From 7c4bdaa3a26f2a8b36f22b1a1257aa11bf136969 Mon Sep 17 00:00:00 2001 From: Thomas Date: Mon, 6 Nov 2023 22:27:45 -0800 Subject: [PATCH 01/94] Core: De-dup props in JdbcUtil (#8992) --- .../apache/iceberg/jdbc/JdbcTableOperations.java | 9 ++++----- .../java/org/apache/iceberg/jdbc/JdbcUtil.java | 16 +++++++--------- 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java index cdc6be6c70a4..6a7d594dd9f6 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java @@ -92,7 +92,7 @@ public void doRefresh() { } } - String newMetadataLocation = table.get(JdbcUtil.METADATA_LOCATION); + String newMetadataLocation = table.get(METADATA_LOCATION_PROP); Preconditions.checkState( newMetadataLocation != null, "Invalid table %s: metadata location is null", @@ -204,7 +204,7 @@ private void createTable(String newMetadataLocation) throws SQLException, Interr } private void validateMetadataLocation(Map table, TableMetadata base) { - String catalogMetadataLocation = table.get(JdbcUtil.METADATA_LOCATION); + String catalogMetadataLocation = table.get(METADATA_LOCATION_PROP); String baseMetadataLocation = base != null ? base.metadataFileLocation() : null; if (!Objects.equals(baseMetadataLocation, catalogMetadataLocation)) { @@ -240,10 +240,9 @@ private Map getTable() table.put(JdbcUtil.CATALOG_NAME, rs.getString(JdbcUtil.CATALOG_NAME)); table.put(JdbcUtil.TABLE_NAMESPACE, rs.getString(JdbcUtil.TABLE_NAMESPACE)); table.put(JdbcUtil.TABLE_NAME, rs.getString(JdbcUtil.TABLE_NAME)); - table.put(JdbcUtil.METADATA_LOCATION, rs.getString(JdbcUtil.METADATA_LOCATION)); + table.put(METADATA_LOCATION_PROP, rs.getString(METADATA_LOCATION_PROP)); table.put( - JdbcUtil.PREVIOUS_METADATA_LOCATION, - rs.getString(JdbcUtil.PREVIOUS_METADATA_LOCATION)); + PREVIOUS_METADATA_LOCATION_PROP, rs.getString(PREVIOUS_METADATA_LOCATION_PROP)); } rs.close(); diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java index 3ffa47d2ea68..1fdf9e221a0a 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java @@ -41,16 +41,14 @@ final class JdbcUtil { static final String CATALOG_NAME = "catalog_name"; static final String TABLE_NAMESPACE = "table_namespace"; static final String TABLE_NAME = "table_name"; - static final String METADATA_LOCATION = "metadata_location"; - static final String PREVIOUS_METADATA_LOCATION = "previous_metadata_location"; static final String DO_COMMIT_SQL = "UPDATE " + CATALOG_TABLE_NAME + " SET " - + METADATA_LOCATION + + JdbcTableOperations.METADATA_LOCATION_PROP + " = ? , " - + PREVIOUS_METADATA_LOCATION + + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + " = ? " + " WHERE " + CATALOG_NAME @@ -59,7 +57,7 @@ final class JdbcUtil { + " = ? AND " + TABLE_NAME + " = ? AND " - + METADATA_LOCATION + + JdbcTableOperations.METADATA_LOCATION_PROP + " = ?"; static final String CREATE_CATALOG_TABLE = "CREATE TABLE " @@ -71,9 +69,9 @@ final class JdbcUtil { + " VARCHAR(255) NOT NULL," + TABLE_NAME + " VARCHAR(255) NOT NULL," - + METADATA_LOCATION + + JdbcTableOperations.METADATA_LOCATION_PROP + " VARCHAR(1000)," - + PREVIOUS_METADATA_LOCATION + + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + " VARCHAR(1000)," + "PRIMARY KEY (" + CATALOG_NAME @@ -164,9 +162,9 @@ final class JdbcUtil { + ", " + TABLE_NAME + ", " - + METADATA_LOCATION + + JdbcTableOperations.METADATA_LOCATION_PROP + ", " - + PREVIOUS_METADATA_LOCATION + + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + ") " + " VALUES (?,?,?,?,null)"; From e8bb8b502c981f6e5dea27cc6f833655bf79f4c6 Mon Sep 17 00:00:00 2001 From: roryqi Date: Wed, 8 Nov 2023 04:55:41 +0800 Subject: [PATCH 02/94] Test: Add a test utility method to programmatically create expected partition specs (#8467) --- .../java/org/apache/iceberg/TestHelpers.java | 40 +++++++++++ .../TestAlterTablePartitionFields.java | 66 ++++++------------- .../source/TestForwardCompatibility.java | 17 +++-- ...tMetadataTablesWithPartitionEvolution.java | 10 +-- .../source/TestSparkMetadataColumns.java | 10 +-- .../TestAlterTablePartitionFields.java | 66 ++++++------------- .../source/TestForwardCompatibility.java | 18 +++-- ...tMetadataTablesWithPartitionEvolution.java | 10 +-- .../source/TestSparkMetadataColumns.java | 10 +-- .../TestAlterTablePartitionFields.java | 66 ++++++------------- .../source/TestForwardCompatibility.java | 18 +++-- ...tMetadataTablesWithPartitionEvolution.java | 10 +-- .../source/TestSparkMetadataColumns.java | 10 +-- .../TestAlterTablePartitionFields.java | 66 ++++++------------- .../source/TestForwardCompatibility.java | 17 +++-- ...tMetadataTablesWithPartitionEvolution.java | 10 +-- .../source/TestSparkMetadataColumns.java | 10 +-- 17 files changed, 214 insertions(+), 240 deletions(-) diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index 153e2de7ea9a..890ae8abd474 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -265,6 +265,10 @@ public static void serialize(final Serializable obj, final OutputStream outputSt } } + public static ExpectedSpecBuilder newExpectedSpecBuilder() { + return new ExpectedSpecBuilder(); + } + public static class KryoHelpers { private KryoHelpers() {} @@ -667,4 +671,40 @@ public List splitOffsets() { return null; } } + + public static class ExpectedSpecBuilder { + private final UnboundPartitionSpec.Builder unboundPartitionSpecBuilder; + + private Schema schema; + + private ExpectedSpecBuilder() { + this.unboundPartitionSpecBuilder = UnboundPartitionSpec.builder(); + } + + public ExpectedSpecBuilder withSchema(Schema newSchema) { + this.schema = newSchema; + return this; + } + + public ExpectedSpecBuilder withSpecId(int newSpecId) { + unboundPartitionSpecBuilder.withSpecId(newSpecId); + return this; + } + + public ExpectedSpecBuilder addField( + String transformAsString, int sourceId, int partitionId, String name) { + unboundPartitionSpecBuilder.addField(transformAsString, sourceId, partitionId, name); + return this; + } + + public ExpectedSpecBuilder addField(String transformAsString, int sourceId, String name) { + unboundPartitionSpecBuilder.addField(transformAsString, sourceId, name); + return this; + } + + public PartitionSpec build() { + Preconditions.checkNotNull(schema, "Field schema is missing"); + return unboundPartitionSpecBuilder.build().bind(schema); + } + } } diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java index 2ecf6b0c4ca7..042e87c729a5 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java @@ -19,9 +19,9 @@ package org.apache.iceberg.spark.extensions; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.connector.catalog.CatalogManager; @@ -392,17 +392,11 @@ public void testReplacePartition() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"ts_hour\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -431,17 +425,11 @@ public void testReplacePartitionAndRename() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"hour_col\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -470,17 +458,11 @@ public void testReplaceNamedPartition() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"ts_hour\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -509,17 +491,11 @@ public void testReplaceNamedPartitionAndRenameDifferently() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"hour_col\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index fe440235901c..96bebf7c8868 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -35,7 +35,6 @@ import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestWriter; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -75,14 +74,18 @@ public class TestForwardCompatibility { // create a spec for the schema that uses a "zero" transform that produces all 0s private static final PartitionSpec UNKNOWN_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 0, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("zero", 1, "id_zero") + .build(); // create a fake spec to use to write table metadata private static final PartitionSpec FAKE_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 0, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"identity\", \"source-id\": 1 } ] }"); + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("identity", 1, "id_zero") + .build(); @Rule public TemporaryFolder temp = new TemporaryFolder(); diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java index a62199181a3a..0baaef1374d4 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java @@ -38,10 +38,10 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -627,9 +627,11 @@ public void testMetadataTablesWithUnknownTransforms() { Table table = validationCatalog.loadTable(tableIdent); PartitionSpec unknownSpec = - PartitionSpecParser.fromJson( - table.schema(), - "{ \"spec-id\": 1, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); // replace the table spec to include an unknown transform TableOperations ops = ((HasTableOperations) table).operations(); diff --git a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index 79b755872dc3..5c7929112fb9 100644 --- a/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -35,11 +35,11 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -75,9 +75,11 @@ public class TestSparkMetadataColumns extends SparkTestBase { Types.NestedField.optional(3, "data", Types.StringType.get())); private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); private static final PartitionSpec UNKNOWN_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 1, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); @Parameterized.Parameters(name = "fileFormat = {0}, vectorized = {1}, formatVersion = {2}") public static Object[][] parameters() { diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java index 0e978e52e570..948fc462de99 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java @@ -19,9 +19,9 @@ package org.apache.iceberg.spark.extensions; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.connector.catalog.CatalogManager; @@ -392,17 +392,11 @@ public void testReplacePartition() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"ts_hour\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -431,17 +425,11 @@ public void testReplacePartitionAndRename() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"hour_col\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -470,17 +458,11 @@ public void testReplaceNamedPartition() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"ts_hour\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -509,17 +491,11 @@ public void testReplaceNamedPartitionAndRenameDifferently() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"hour_col\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index fe440235901c..6ab9e57949e3 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -35,7 +35,6 @@ import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestWriter; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -75,14 +74,19 @@ public class TestForwardCompatibility { // create a spec for the schema that uses a "zero" transform that produces all 0s private static final PartitionSpec UNKNOWN_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 0, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("zero", 1, "id_zero") + .build(); + // create a fake spec to use to write table metadata private static final PartitionSpec FAKE_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 0, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"identity\", \"source-id\": 1 } ] }"); + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("identity", 1, "id_zero") + .build(); @Rule public TemporaryFolder temp = new TemporaryFolder(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java index a62199181a3a..0baaef1374d4 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java @@ -38,10 +38,10 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -627,9 +627,11 @@ public void testMetadataTablesWithUnknownTransforms() { Table table = validationCatalog.loadTable(tableIdent); PartitionSpec unknownSpec = - PartitionSpecParser.fromJson( - table.schema(), - "{ \"spec-id\": 1, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); // replace the table spec to include an unknown transform TableOperations ops = ((HasTableOperations) table).operations(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index 79b755872dc3..5c7929112fb9 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -35,11 +35,11 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -75,9 +75,11 @@ public class TestSparkMetadataColumns extends SparkTestBase { Types.NestedField.optional(3, "data", Types.StringType.get())); private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); private static final PartitionSpec UNKNOWN_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 1, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); @Parameterized.Parameters(name = "fileFormat = {0}, vectorized = {1}, formatVersion = {2}") public static Object[][] parameters() { diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java index 0e978e52e570..948fc462de99 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java @@ -19,9 +19,9 @@ package org.apache.iceberg.spark.extensions; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.connector.catalog.CatalogManager; @@ -392,17 +392,11 @@ public void testReplacePartition() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"ts_hour\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -431,17 +425,11 @@ public void testReplacePartitionAndRename() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"hour_col\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -470,17 +458,11 @@ public void testReplaceNamedPartition() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"ts_hour\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -509,17 +491,11 @@ public void testReplaceNamedPartitionAndRenameDifferently() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"hour_col\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 73e572ecaecd..80a8196f8a09 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -34,7 +34,6 @@ import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestWriter; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -75,14 +74,19 @@ public class TestForwardCompatibility { // create a spec for the schema that uses a "zero" transform that produces all 0s private static final PartitionSpec UNKNOWN_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 0, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("zero", 1, "id_zero") + .build(); + // create a fake spec to use to write table metadata private static final PartitionSpec FAKE_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 0, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"identity\", \"source-id\": 1 } ] }"); + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("identity", 1, "id_zero") + .build(); @Rule public TemporaryFolder temp = new TemporaryFolder(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java index b4b60dc02dd5..ea65fead10fc 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java @@ -37,10 +37,10 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -627,9 +627,11 @@ public void testMetadataTablesWithUnknownTransforms() { Table table = validationCatalog.loadTable(tableIdent); PartitionSpec unknownSpec = - PartitionSpecParser.fromJson( - table.schema(), - "{ \"spec-id\": 1, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); // replace the table spec to include an unknown transform TableOperations ops = ((HasTableOperations) table).operations(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index 8a2fae811750..b2361c188c17 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -34,11 +34,11 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -75,9 +75,11 @@ public class TestSparkMetadataColumns extends SparkTestBase { Types.NestedField.optional(3, "data", Types.StringType.get())); private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); private static final PartitionSpec UNKNOWN_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 1, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); @Parameterized.Parameters(name = "fileFormat = {0}, vectorized = {1}, formatVersion = {2}") public static Object[][] parameters() { diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java index 0e978e52e570..948fc462de99 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java @@ -19,9 +19,9 @@ package org.apache.iceberg.spark.extensions; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.source.SparkTable; import org.apache.spark.sql.connector.catalog.CatalogManager; @@ -392,17 +392,11 @@ public void testReplacePartition() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"ts_hour\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -431,17 +425,11 @@ public void testReplacePartitionAndRename() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"hour_col\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -470,17 +458,11 @@ public void testReplaceNamedPartition() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"ts_hour\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "ts_hour") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); @@ -509,17 +491,11 @@ public void testReplaceNamedPartitionAndRenameDifferently() { .build(); } else { expected = - PartitionSpecParser.fromJson( - table.schema(), - "{\n" - + " \"spec-id\" : 2,\n" - + " \"fields\" : [ {\n" - + " \"name\" : \"hour_col\",\n" - + " \"transform\" : \"hour\",\n" - + " \"source-id\" : 3,\n" - + " \"field-id\" : 1001\n" - + " } ]\n" - + "}"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(2) + .addField("hour", 3, 1001, "hour_col") + .build(); } Assert.assertEquals( "Should changed from daily to hourly partitioned field", expected, table.spec()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 73e572ecaecd..446989d1af3e 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -34,7 +34,6 @@ import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestWriter; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -75,14 +74,18 @@ public class TestForwardCompatibility { // create a spec for the schema that uses a "zero" transform that produces all 0s private static final PartitionSpec UNKNOWN_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 0, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("zero", 1, "id_zero") + .build(); // create a fake spec to use to write table metadata private static final PartitionSpec FAKE_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 0, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"identity\", \"source-id\": 1 } ] }"); + org.apache.iceberg.TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(0) + .addField("identity", 1, "id_zero") + .build(); @Rule public TemporaryFolder temp = new TemporaryFolder(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java index b4b60dc02dd5..ea65fead10fc 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java @@ -37,10 +37,10 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -627,9 +627,11 @@ public void testMetadataTablesWithUnknownTransforms() { Table table = validationCatalog.loadTable(tableIdent); PartitionSpec unknownSpec = - PartitionSpecParser.fromJson( - table.schema(), - "{ \"spec-id\": 1, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(table.schema()) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); // replace the table spec to include an unknown transform TableOperations ops = ((HasTableOperations) table).operations(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index 8a2fae811750..b2361c188c17 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -34,11 +34,11 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -75,9 +75,11 @@ public class TestSparkMetadataColumns extends SparkTestBase { Types.NestedField.optional(3, "data", Types.StringType.get())); private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); private static final PartitionSpec UNKNOWN_SPEC = - PartitionSpecParser.fromJson( - SCHEMA, - "{ \"spec-id\": 1, \"fields\": [ { \"name\": \"id_zero\", \"transform\": \"zero\", \"source-id\": 1 } ] }"); + TestHelpers.newExpectedSpecBuilder() + .withSchema(SCHEMA) + .withSpecId(1) + .addField("zero", 1, "id_zero") + .build(); @Parameterized.Parameters(name = "fileFormat = {0}, vectorized = {1}, formatVersion = {2}") public static Object[][] parameters() { From 6105375d47f9d8b80dcbba72302e70b3f370642c Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 7 Nov 2023 23:00:44 -0800 Subject: [PATCH 03/94] Infra: Add 1.4.2 as latest release to issue template (#9001) --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index 4d553117e67a..0e41cfb7c9a4 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -9,7 +9,8 @@ body: description: What Apache Iceberg version are you using? multiple: false options: - - "1.4.1 (latest release)" + - "1.4.2 (latest release)" + - "1.4.1" - "1.4.0" - "1.3.1" - "1.3.0" From 1fb8e4fbd11dea3c16e7c16d56d543bee2b53464 Mon Sep 17 00:00:00 2001 From: Wonjae Lee <38933452+leewjae@users.noreply.github.com> Date: Wed, 8 Nov 2023 18:41:44 +0900 Subject: [PATCH 04/94] Core: Add a constructor to StaticTableOperations (#8996) --- .../org/apache/iceberg/StaticTableOperations.java | 12 ++++++++++++ .../apache/iceberg/util/TestReachableFileUtil.java | 2 +- .../iceberg/spark/actions/BaseSparkAction.java | 5 ++--- 3 files changed, 15 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/StaticTableOperations.java b/core/src/main/java/org/apache/iceberg/StaticTableOperations.java index fada6de9c487..77ee0920edc9 100644 --- a/core/src/main/java/org/apache/iceberg/StaticTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/StaticTableOperations.java @@ -44,6 +44,18 @@ public StaticTableOperations( this.locationProvider = locationProvider; } + public StaticTableOperations(TableMetadata staticMetadata, FileIO io) { + this(staticMetadata, io, null); + } + + public StaticTableOperations( + TableMetadata staticMetadata, FileIO io, LocationProvider locationProvider) { + this.staticMetadata = staticMetadata; + this.metadataFileLocation = staticMetadata.metadataFileLocation(); + this.io = io; + this.locationProvider = locationProvider; + } + @Override public TableMetadata current() { if (staticMetadata == null) { diff --git a/core/src/test/java/org/apache/iceberg/util/TestReachableFileUtil.java b/core/src/test/java/org/apache/iceberg/util/TestReachableFileUtil.java index b8d686246b0f..13ed75f017ee 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestReachableFileUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestReachableFileUtil.java @@ -124,7 +124,7 @@ public void testVersionHintWithStaticTables() { TableMetadata metadata = ops.current(); String metadataFileLocation = metadata.metadataFileLocation(); - StaticTableOperations staticOps = new StaticTableOperations(metadataFileLocation, table.io()); + StaticTableOperations staticOps = new StaticTableOperations(metadata, table.io()); Table staticTable = new BaseTable(staticOps, metadataFileLocation); String reportedVersionHintLocation = ReachableFileUtil.versionHintLocation(staticTable); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java index 62f51675265e..d0e71a707db9 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseSparkAction.java @@ -137,9 +137,8 @@ protected JobGroupInfo newJobGroupInfo(String groupId, String desc) { } protected Table newStaticTable(TableMetadata metadata, FileIO io) { - String metadataFileLocation = metadata.metadataFileLocation(); - StaticTableOperations ops = new StaticTableOperations(metadataFileLocation, io); - return new BaseTable(ops, metadataFileLocation); + StaticTableOperations ops = new StaticTableOperations(metadata, io); + return new BaseTable(ops, metadata.metadataFileLocation()); } protected Dataset contentFileDS(Table table) { From e8cf33db7d3fc637504a51a801c055dce54474b7 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Wed, 8 Nov 2023 19:40:02 +0800 Subject: [PATCH 05/94] Docs: Add note that snapshot expiration and cleanup orphan files could corrupt Flink job state (#9002) --- docs/flink-writes.md | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/flink-writes.md b/docs/flink-writes.md index 641fa09e3c7f..e078a8286849 100644 --- a/docs/flink-writes.md +++ b/docs/flink-writes.md @@ -270,4 +270,13 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ ... ``` -Check out all the options here: [write-options](/flink-configuration#write-options) \ No newline at end of file +Check out all the options here: [write-options](/flink-configuration#write-options) + +## Notes + +Flink streaming write jobs rely on snapshot summary to keep the last committed checkpoint ID, and +store uncommitted data as temporary files. Therefore, [expiring snapshots](../tables/maintenance#expire-snapshots) +and [deleting orphan files](../tables/maintenance#delete-orphan-files) could possibly corrupt +the state of the Flink job. To avoid that, make sure to keep the last snapshot created by the Flink +job (which can be identified by the `flink.job-id` property in the summary), and only delete +orphan files that are old enough. \ No newline at end of file From af132c7f8d0e820a3bdc23de4dd76f343c7bb399 Mon Sep 17 00:00:00 2001 From: Jacob Marble Date: Wed, 8 Nov 2023 12:51:38 -0800 Subject: [PATCH 06/94] Spec: Clarify ns timestamps for ORC deserialization (#9007) Helps #8657 In order for ORC types `timestamp` and `timestamp_instant` to be correctly converted to Iceberg `timestamp`, `timestamp_ns`, `timestamptz`, and `timestamptz_ns`, we need an ORC type attribute. --- format/spec.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/format/spec.md b/format/spec.md index 9a36cd9f3dea..27e2762f7724 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1000,10 +1000,10 @@ Lists must use the [3-level representation](https://github.com/apache/parquet-fo | **`decimal(P,S)`** | `decimal` | | | | **`date`** | `date` | | | | **`time`** | `long` | `iceberg.long-type`=`TIME` | Stores microseconds from midnight. | -| **`timestamp`** | `timestamp` | | Stores microseconds from 2015-01-01 00:00:00.000000. [1], [2] | -| **`timestamptz`** | `timestamp_instant` | | Stores microseconds from 2015-01-01 00:00:00.000000 UTC. [1], [2] | -| **`timestamp_ns`** | `timestamp` | | Stores nanoseconds from 2015-01-01 00:00:00.000000000. [1] | -| **`timestamptz_ns`** | `timestamp_instant` | | Stores nanoseconds from 2015-01-01 00:00:00.000000000 UTC. [1] | +| **`timestamp`** | `timestamp` | `iceberg.timestamp-unit`=`MICROS` | Stores microseconds from 2015-01-01 00:00:00.000000. [1], [2] | +| **`timestamptz`** | `timestamp_instant` | `iceberg.timestamp-unit`=`MICROS` | Stores microseconds from 2015-01-01 00:00:00.000000 UTC. [1], [2] | +| **`timestamp_ns`** | `timestamp` | `iceberg.timestamp-unit`=`NANOS` | Stores nanoseconds from 2015-01-01 00:00:00.000000000. [1] | +| **`timestamptz_ns`** | `timestamp_instant` | `iceberg.timestamp-unit`=`NANOS` | Stores nanoseconds from 2015-01-01 00:00:00.000000000 UTC. [1] | | **`string`** | `string` | | ORC `varchar` and `char` would also map to **`string`**. | | **`uuid`** | `binary` | `iceberg.binary-type`=`UUID` | | | **`fixed(L)`** | `binary` | `iceberg.binary-type`=`FIXED` & `iceberg.length`=`L` | The length would not be checked by the ORC reader and should be checked by the adapter. | @@ -1015,7 +1015,7 @@ Lists must use the [3-level representation](https://github.com/apache/parquet-fo Notes: 1. ORC's [TimestampColumnVector](https://orc.apache.org/api/hive-storage-api/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.html) consists of a time field (milliseconds since epoch) and a nanos field (nanoseconds within the second). Hence the milliseconds within the second are reported twice; once in the time field and again in the nanos field. The read adapter should only use milliseconds within the second from one of these fields. The write adapter should also report milliseconds within the second twice; once in the time field and again in the nanos field. ORC writer is expected to correctly consider millis information from one of the fields. More details at https://issues.apache.org/jira/browse/ORC-546 -2. ORC `timestamp` and `timestamp_instant` values store nanosecond precision. Iceberg ORC writers for Iceberg types `timestamp` and `timestamptz` **must** truncate nanoseconds to microseconds. +2. ORC `timestamp` and `timestamp_instant` values store nanosecond precision. Iceberg ORC writers for Iceberg types `timestamp` and `timestamptz` **must** truncate nanoseconds to microseconds. `iceberg.timestamp-unit` is assumed to be `MICROS` if not present. One of the interesting challenges with this is how to map Iceberg’s schema evolution (id based) on to ORC’s (name based). In theory, we could use Iceberg’s column ids as the column and field names, but that would be inconvenient. From f00d3094a1728b60dba96fc251707dd08d8a6310 Mon Sep 17 00:00:00 2001 From: Karuppayya Date: Wed, 8 Nov 2023 16:31:28 -0800 Subject: [PATCH 07/94] Spark 3.4: Display more read metrics on Spark SQL UI (#9009) This change cherry-picks PR #8717 to Spark 3.4. --- .../iceberg/spark/source/SparkScan.java | 83 ++++++++++++-- .../source/metrics/EqualityDeleteFiles.java | 36 ++++++ .../source/metrics/IndexedDeleteFiles.java | 36 ++++++ .../source/metrics/PositionalDeleteFiles.java | 36 ++++++ ...nedDataFiles.java => ResultDataFiles.java} | 6 +- .../source/metrics/ResultDeleteFiles.java | 36 ++++++ .../metrics/ScannedDeleteManifests.java | 36 ++++++ .../source/metrics/SkippedDeleteFiles.java | 36 ++++++ .../metrics/SkippedDeleteManifests.java | 36 ++++++ .../metrics/TaskEqualityDeleteFiles.java | 47 ++++++++ .../metrics/TaskIndexedDeleteFiles.java | 47 ++++++++ .../metrics/TaskPositionalDeleteFiles.java | 47 ++++++++ ...ataFiles.java => TaskResultDataFiles.java} | 10 +- .../source/metrics/TaskResultDeleteFiles.java | 47 ++++++++ .../metrics/TaskScannedDeleteManifests.java | 47 ++++++++ .../metrics/TaskSkippedDeleteFiles.java | 47 ++++++++ .../metrics/TaskSkippedDeleteManifests.java | 47 ++++++++ ...leSize.java => TaskTotalDataFileSize.java} | 10 +- .../metrics/TaskTotalDataManifests.java | 47 ++++++++ .../metrics/TaskTotalDeleteFileSize.java | 48 ++++++++ .../metrics/TaskTotalDeleteManifests.java | 47 ++++++++ .../source/metrics/TotalDataFileSize.java | 36 ++++++ ...lFileSize.java => TotalDataManifests.java} | 6 +- .../source/metrics/TotalDeleteFileSize.java | 36 ++++++ .../source/metrics/TotalDeleteManifests.java | 36 ++++++ .../spark/source/TestSparkReadMetrics.java | 106 ++++++++++++++++-- 26 files changed, 1017 insertions(+), 35 deletions(-) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java rename spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/{ScannedDataFiles.java => ResultDataFiles.java} (87%) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java rename spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/{TaskScannedDataFiles.java => TaskResultDataFiles.java} (83%) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java rename spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/{TaskTotalFileSize.java => TaskTotalDataFileSize.java} (83%) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java rename spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/{TotalFileSize.java => TotalDataManifests.java} (87%) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java index 6c3b2db14367..6efe8a080bde 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkScan.java @@ -35,19 +35,39 @@ import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkReadConf; import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.source.metrics.EqualityDeleteFiles; +import org.apache.iceberg.spark.source.metrics.IndexedDeleteFiles; import org.apache.iceberg.spark.source.metrics.NumDeletes; import org.apache.iceberg.spark.source.metrics.NumSplits; -import org.apache.iceberg.spark.source.metrics.ScannedDataFiles; +import org.apache.iceberg.spark.source.metrics.PositionalDeleteFiles; +import org.apache.iceberg.spark.source.metrics.ResultDataFiles; +import org.apache.iceberg.spark.source.metrics.ResultDeleteFiles; import org.apache.iceberg.spark.source.metrics.ScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.ScannedDeleteManifests; import org.apache.iceberg.spark.source.metrics.SkippedDataFiles; import org.apache.iceberg.spark.source.metrics.SkippedDataManifests; -import org.apache.iceberg.spark.source.metrics.TaskScannedDataFiles; +import org.apache.iceberg.spark.source.metrics.SkippedDeleteFiles; +import org.apache.iceberg.spark.source.metrics.SkippedDeleteManifests; +import org.apache.iceberg.spark.source.metrics.TaskEqualityDeleteFiles; +import org.apache.iceberg.spark.source.metrics.TaskIndexedDeleteFiles; +import org.apache.iceberg.spark.source.metrics.TaskPositionalDeleteFiles; +import org.apache.iceberg.spark.source.metrics.TaskResultDataFiles; +import org.apache.iceberg.spark.source.metrics.TaskResultDeleteFiles; import org.apache.iceberg.spark.source.metrics.TaskScannedDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskScannedDeleteManifests; import org.apache.iceberg.spark.source.metrics.TaskSkippedDataFiles; import org.apache.iceberg.spark.source.metrics.TaskSkippedDataManifests; -import org.apache.iceberg.spark.source.metrics.TaskTotalFileSize; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDeleteFiles; +import org.apache.iceberg.spark.source.metrics.TaskSkippedDeleteManifests; +import org.apache.iceberg.spark.source.metrics.TaskTotalDataFileSize; +import org.apache.iceberg.spark.source.metrics.TaskTotalDataManifests; +import org.apache.iceberg.spark.source.metrics.TaskTotalDeleteFileSize; +import org.apache.iceberg.spark.source.metrics.TaskTotalDeleteManifests; import org.apache.iceberg.spark.source.metrics.TaskTotalPlanningDuration; -import org.apache.iceberg.spark.source.metrics.TotalFileSize; +import org.apache.iceberg.spark.source.metrics.TotalDataFileSize; +import org.apache.iceberg.spark.source.metrics.TotalDataManifests; +import org.apache.iceberg.spark.source.metrics.TotalDeleteFileSize; +import org.apache.iceberg.spark.source.metrics.TotalDeleteManifests; import org.apache.iceberg.spark.source.metrics.TotalPlanningDuration; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; @@ -200,12 +220,32 @@ public CustomTaskMetric[] reportDriverMetrics() { } List driverMetrics = Lists.newArrayList(); - driverMetrics.add(TaskTotalFileSize.from(scanReport)); + + // common driverMetrics.add(TaskTotalPlanningDuration.from(scanReport)); - driverMetrics.add(TaskSkippedDataFiles.from(scanReport)); - driverMetrics.add(TaskScannedDataFiles.from(scanReport)); - driverMetrics.add(TaskSkippedDataManifests.from(scanReport)); + + // data manifests + driverMetrics.add(TaskTotalDataManifests.from(scanReport)); driverMetrics.add(TaskScannedDataManifests.from(scanReport)); + driverMetrics.add(TaskSkippedDataManifests.from(scanReport)); + + // data files + driverMetrics.add(TaskResultDataFiles.from(scanReport)); + driverMetrics.add(TaskSkippedDataFiles.from(scanReport)); + driverMetrics.add(TaskTotalDataFileSize.from(scanReport)); + + // delete manifests + driverMetrics.add(TaskTotalDeleteManifests.from(scanReport)); + driverMetrics.add(TaskScannedDeleteManifests.from(scanReport)); + driverMetrics.add(TaskSkippedDeleteManifests.from(scanReport)); + + // delete files + driverMetrics.add(TaskTotalDeleteFileSize.from(scanReport)); + driverMetrics.add(TaskResultDeleteFiles.from(scanReport)); + driverMetrics.add(TaskEqualityDeleteFiles.from(scanReport)); + driverMetrics.add(TaskIndexedDeleteFiles.from(scanReport)); + driverMetrics.add(TaskPositionalDeleteFiles.from(scanReport)); + driverMetrics.add(TaskSkippedDeleteFiles.from(scanReport)); return driverMetrics.toArray(new CustomTaskMetric[0]); } @@ -213,14 +253,35 @@ public CustomTaskMetric[] reportDriverMetrics() { @Override public CustomMetric[] supportedCustomMetrics() { return new CustomMetric[] { + // task metrics new NumSplits(), new NumDeletes(), - new TotalFileSize(), + + // common new TotalPlanningDuration(), + + // data manifests + new TotalDataManifests(), new ScannedDataManifests(), new SkippedDataManifests(), - new ScannedDataFiles(), - new SkippedDataFiles() + + // data files + new ResultDataFiles(), + new SkippedDataFiles(), + new TotalDataFileSize(), + + // delete manifests + new TotalDeleteManifests(), + new ScannedDeleteManifests(), + new SkippedDeleteManifests(), + + // delete files + new TotalDeleteFileSize(), + new ResultDeleteFiles(), + new EqualityDeleteFiles(), + new IndexedDeleteFiles(), + new PositionalDeleteFiles(), + new SkippedDeleteFiles() }; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java new file mode 100644 index 000000000000..754145f7d252 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/EqualityDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class EqualityDeleteFiles extends CustomSumMetric { + + static final String NAME = "equalityDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of equality delete files"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java new file mode 100644 index 000000000000..7fc5b9066cdc --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/IndexedDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class IndexedDeleteFiles extends CustomSumMetric { + + static final String NAME = "indexedDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of indexed delete files"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java new file mode 100644 index 000000000000..5de75776ea4f --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/PositionalDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class PositionalDeleteFiles extends CustomSumMetric { + + static final String NAME = "positionalDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of positional delete files"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java similarity index 87% rename from spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataFiles.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java index f453872fdc29..21959cbf6c63 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDataFiles.java @@ -20,9 +20,9 @@ import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class ScannedDataFiles extends CustomSumMetric { +public class ResultDataFiles extends CustomSumMetric { - static final String NAME = "scannedDataFiles"; + static final String NAME = "resultDataFiles"; @Override public String name() { @@ -31,6 +31,6 @@ public String name() { @Override public String description() { - return "number of scanned data files"; + return "number of result data files"; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java new file mode 100644 index 000000000000..9c6ad2ca328a --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ResultDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class ResultDeleteFiles extends CustomSumMetric { + + static final String NAME = "resultDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of result delete files"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java new file mode 100644 index 000000000000..1fa006b7b193 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/ScannedDeleteManifests.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class ScannedDeleteManifests extends CustomSumMetric { + + static final String NAME = "scannedDeleteManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of scanned delete manifests"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java new file mode 100644 index 000000000000..70597be67113 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteFiles.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class SkippedDeleteFiles extends CustomSumMetric { + + static final String NAME = "skippedDeleteFiles"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of skipped delete files"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java new file mode 100644 index 000000000000..0336170b45a1 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/SkippedDeleteManifests.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class SkippedDeleteManifests extends CustomSumMetric { + + static final String NAME = "skippedDeleteManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "number of skipped delete manifest"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java new file mode 100644 index 000000000000..ecd14bcca31d --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskEqualityDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskEqualityDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskEqualityDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return EqualityDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskEqualityDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().equalityDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskEqualityDeleteFiles(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java new file mode 100644 index 000000000000..63b6767e955d --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskIndexedDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskIndexedDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskIndexedDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return IndexedDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskIndexedDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().indexedDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskIndexedDeleteFiles(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java new file mode 100644 index 000000000000..805f22bf0d7c --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskPositionalDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskPositionalDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskPositionalDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return PositionalDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskPositionalDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().positionalDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskPositionalDeleteFiles(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java similarity index 83% rename from spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java index d9a527da08f6..a27142131403 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDataFiles.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDataFiles.java @@ -22,16 +22,16 @@ import org.apache.iceberg.metrics.ScanReport; import org.apache.spark.sql.connector.metric.CustomTaskMetric; -public class TaskScannedDataFiles implements CustomTaskMetric { +public class TaskResultDataFiles implements CustomTaskMetric { private final long value; - private TaskScannedDataFiles(long value) { + private TaskResultDataFiles(long value) { this.value = value; } @Override public String name() { - return ScannedDataFiles.NAME; + return ResultDataFiles.NAME; } @Override @@ -39,9 +39,9 @@ public long value() { return value; } - public static TaskScannedDataFiles from(ScanReport scanReport) { + public static TaskResultDataFiles from(ScanReport scanReport) { CounterResult counter = scanReport.scanMetrics().resultDataFiles(); long value = counter != null ? counter.value() : 0L; - return new TaskScannedDataFiles(value); + return new TaskResultDataFiles(value); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java new file mode 100644 index 000000000000..aea8ca07dd05 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskResultDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskResultDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskResultDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return ResultDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskResultDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().resultDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskResultDeleteFiles(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java new file mode 100644 index 000000000000..1766cf2f6835 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskScannedDeleteManifests.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskScannedDeleteManifests implements CustomTaskMetric { + private final long value; + + private TaskScannedDeleteManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return ScannedDeleteManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskScannedDeleteManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().scannedDeleteManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskScannedDeleteManifests(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java new file mode 100644 index 000000000000..87579751742c --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteFiles.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDeleteFiles implements CustomTaskMetric { + private final long value; + + private TaskSkippedDeleteFiles(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDeleteFiles.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskSkippedDeleteFiles from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDeleteFiles(); + long value = counter != null ? counter.value() : 0L; + return new TaskSkippedDeleteFiles(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java new file mode 100644 index 000000000000..4a9c71e0c1e4 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskSkippedDeleteManifests.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskSkippedDeleteManifests implements CustomTaskMetric { + private final long value; + + private TaskSkippedDeleteManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return SkippedDeleteManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskSkippedDeleteManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().skippedDeleteManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskSkippedDeleteManifests(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java similarity index 83% rename from spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java index c300d835e777..3f5a224425d8 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataFileSize.java @@ -22,17 +22,17 @@ import org.apache.iceberg.metrics.ScanReport; import org.apache.spark.sql.connector.metric.CustomTaskMetric; -public class TaskTotalFileSize implements CustomTaskMetric { +public class TaskTotalDataFileSize implements CustomTaskMetric { private final long value; - private TaskTotalFileSize(long value) { + private TaskTotalDataFileSize(long value) { this.value = value; } @Override public String name() { - return TotalFileSize.NAME; + return TotalDataFileSize.NAME; } @Override @@ -40,9 +40,9 @@ public long value() { return value; } - public static TaskTotalFileSize from(ScanReport scanReport) { + public static TaskTotalDataFileSize from(ScanReport scanReport) { CounterResult counter = scanReport.scanMetrics().totalFileSizeInBytes(); long value = counter != null ? counter.value() : 0L; - return new TaskTotalFileSize(value); + return new TaskTotalDataFileSize(value); } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java new file mode 100644 index 000000000000..6d8c3c24e460 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDataManifests.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalDataManifests implements CustomTaskMetric { + private final long value; + + private TaskTotalDataManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalDataManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalDataManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().totalDataManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskTotalDataManifests(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java new file mode 100644 index 000000000000..17ecec78da3f --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteFileSize.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalDeleteFileSize implements CustomTaskMetric { + + private final long value; + + private TaskTotalDeleteFileSize(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalDeleteFileSize.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalDeleteFileSize from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().totalDeleteFileSizeInBytes(); + long value = counter != null ? counter.value() : 0L; + return new TaskTotalDeleteFileSize(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java new file mode 100644 index 000000000000..ff55c1be89e3 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TaskTotalDeleteManifests.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.iceberg.metrics.CounterResult; +import org.apache.iceberg.metrics.ScanReport; +import org.apache.spark.sql.connector.metric.CustomTaskMetric; + +public class TaskTotalDeleteManifests implements CustomTaskMetric { + private final long value; + + private TaskTotalDeleteManifests(long value) { + this.value = value; + } + + @Override + public String name() { + return TotalDeleteManifests.NAME; + } + + @Override + public long value() { + return value; + } + + public static TaskTotalDeleteManifests from(ScanReport scanReport) { + CounterResult counter = scanReport.scanMetrics().totalDeleteManifests(); + long value = counter != null ? counter.value() : 0L; + return new TaskTotalDeleteManifests(value); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java new file mode 100644 index 000000000000..b1ff8a46368c --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataFileSize.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalDataFileSize extends CustomSumMetric { + + static final String NAME = "totalDataFileSize"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total data file size (bytes)"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java similarity index 87% rename from spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java rename to spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java index 994626e54f10..de8f04be7767 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalFileSize.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDataManifests.java @@ -20,9 +20,9 @@ import org.apache.spark.sql.connector.metric.CustomSumMetric; -public class TotalFileSize extends CustomSumMetric { +public class TotalDataManifests extends CustomSumMetric { - static final String NAME = "totalFileSize"; + static final String NAME = "totalDataManifest"; @Override public String name() { @@ -31,6 +31,6 @@ public String name() { @Override public String description() { - return "total file size (bytes)"; + return "total data manifests"; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java new file mode 100644 index 000000000000..da4303325273 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteFileSize.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalDeleteFileSize extends CustomSumMetric { + + static final String NAME = "totalDeleteFileSize"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total delete file size (bytes)"; + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java new file mode 100644 index 000000000000..7442dfdb6ffb --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/metrics/TotalDeleteManifests.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.source.metrics; + +import org.apache.spark.sql.connector.metric.CustomSumMetric; + +public class TotalDeleteManifests extends CustomSumMetric { + + static final String NAME = "totalDeleteManifests"; + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "total delete manifests"; + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java index 7b943372d167..fea5a1d3e1c3 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -43,7 +43,7 @@ public void removeTables() { @Test public void testReadMetricsForV1Table() throws NoSuchTableException { sql( - "CREATE TABLE %s (id BIGINT) USING iceberg TBLPROPERTIES ('format-version'='2')", + "CREATE TABLE %s (id BIGINT) USING iceberg TBLPROPERTIES ('format-version'='1')", tableName); spark.range(10000).coalesce(1).writeTo(tableName).append(); @@ -56,12 +56,31 @@ public void testReadMetricsForV1Table() throws NoSuchTableException { seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); Map metricsMap = JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); - Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + // Common + Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + + // data manifests + Assertions.assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("scannedDataFiles").value()).isEqualTo(1); Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("totalFileSize").value()).isNotEqualTo(0); - Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + + // data files + Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); + + // delete manifests + Assertions.assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); + + // delete files + Assertions.assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); } @Test @@ -80,11 +99,82 @@ public void testReadMetricsForV2Table() throws NoSuchTableException { seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); Map metricsMap = JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); - Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + + // Common + Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + + // data manifests + Assertions.assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("scannedDataFiles").value()).isEqualTo(1); Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("totalFileSize").value()).isNotEqualTo(0); + + // data files + Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); + + // delete manifests + Assertions.assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); + + // delete files + Assertions.assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); + } + + @Test + public void testDeleteMetrics() throws NoSuchTableException { + sql( + "CREATE TABLE %s (id BIGINT)" + + " USING iceberg" + + " TBLPROPERTIES (\n" + + " 'write.delete.mode'='merge-on-read',\n" + + " 'write.update.mode'='merge-on-read',\n" + + " 'write.merge.mode'='merge-on-read',\n" + + " 'format-version'='2'\n" + + " )", + tableName); + + spark.range(10000).coalesce(1).writeTo(tableName).append(); + + spark.sql(String.format("DELETE FROM %s WHERE id = 1", tableName)).collect(); + Dataset df = spark.sql(String.format("SELECT * FROM %s", tableName)); + df.collect(); + + List sparkPlans = + seqAsJavaListConverter(df.queryExecution().executedPlan().collectLeaves()).asJava(); + Map metricsMap = + JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); + + // Common Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + + // data manifests + Assertions.assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + + // data files + Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); + + // delete manifests + Assertions.assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); + + // delete files + Assertions.assertThat(metricsMap.get("totalDeleteFileSize").value()).isNotEqualTo(0); + Assertions.assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); + Assertions.assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(1); + Assertions.assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); } } From 175a7fb7a00af30e68fd217947f5342fe337d6b8 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 9 Nov 2023 15:44:20 +0100 Subject: [PATCH 08/94] Core: Use InMemoryCatalog as backend catalog (#9014) --- .../apache/iceberg/rest/TestRESTCatalog.java | 103 ++++++++++++++---- 1 file changed, 80 insertions(+), 23 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index fe708bde1147..3de9c7b1d3f7 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -52,7 +52,6 @@ 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.CatalogTests; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SessionCatalog; @@ -63,8 +62,8 @@ import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.ServiceFailureException; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.inmemory.InMemoryCatalog; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.jdbc.JdbcCatalog; import org.apache.iceberg.metrics.MetricsReport; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -105,7 +104,7 @@ public class TestRESTCatalog extends CatalogTests { @TempDir public Path temp; private RESTCatalog restCatalog; - private JdbcCatalog backendCatalog; + private InMemoryCatalog backendCatalog; private Server httpServer; @BeforeEach @@ -113,19 +112,10 @@ public void createCatalog() throws Exception { File warehouse = temp.toFile(); Configuration conf = new Configuration(); - this.backendCatalog = new JdbcCatalog(); - backendCatalog.setConf(conf); - Map backendCatalogProperties = - ImmutableMap.of( - CatalogProperties.WAREHOUSE_LOCATION, - warehouse.getAbsolutePath(), - CatalogProperties.URI, - "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", ""), - JdbcCatalog.PROPERTY_PREFIX + "username", - "user", - JdbcCatalog.PROPERTY_PREFIX + "password", - "password"); - backendCatalog.initialize("backend", backendCatalogProperties); + this.backendCatalog = new InMemoryCatalog(); + this.backendCatalog.initialize( + "in-memory", + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse.getAbsolutePath())); Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=catalog"); @@ -247,6 +237,11 @@ protected boolean supportsNestedNamespaces() { return true; } + @Override + protected boolean requiresNamespaceCreate() { + return true; + } + /* RESTCatalog specific tests */ @Test @@ -783,6 +778,10 @@ public void testTableSnapshotLoading() { "snapshot-loading-mode", "refs")); + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + // Create a table with multiple snapshots Table table = catalog.createTable(TABLE, SCHEMA); table @@ -877,6 +876,10 @@ public void testTableSnapshotLoadingWithDivergedBranches(String formatVersion) { "snapshot-loading-mode", "refs")); + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + Table table = catalog.createTable( TABLE, @@ -997,6 +1000,10 @@ public void lazySnapshotLoadingWithDivergedHistory() { "snapshot-loading-mode", "refs")); + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + Table table = catalog.createTable(TABLE, SCHEMA, PartitionSpec.unpartitioned(), ImmutableMap.of()); @@ -1105,6 +1112,10 @@ public void testTableAuth( required(1, "id", Types.IntegerType.get(), "unique ID"), required(2, "data", Types.StringType.get())); + if (requiresNamespaceCreate()) { + catalog.createNamespace(ident.namespace()); + } + Table table = catalog.createTable(ident, expectedSchema); Assertions.assertThat(table.schema().asStruct()) .as("Schema should match") @@ -1428,6 +1439,10 @@ public void testCatalogWithCustomMetricsReporter() throws IOException { CatalogProperties.METRICS_REPORTER_IMPL, CustomMetricsReporter.class.getName())); + if (requiresNamespaceCreate()) { + restCatalog.createNamespace(TABLE.namespace()); + } + restCatalog.buildTable(TABLE, SCHEMA).create(); Table table = restCatalog.loadTable(TABLE); table @@ -1951,6 +1966,10 @@ public void diffAgainstSingleTable() { Namespace namespace = Namespace.of("namespace"); TableIdentifier identifier = TableIdentifier.of(namespace, "multipleDiffsAgainstSingleTable"); + if (requiresNamespaceCreate()) { + catalog().createNamespace(namespace); + } + Table table = catalog().buildTable(identifier, SCHEMA).create(); Transaction transaction = table.newTransaction(); @@ -1983,6 +2002,10 @@ public void multipleDiffsAgainstMultipleTables() { TableIdentifier identifier1 = TableIdentifier.of(namespace, "multiDiffTable1"); TableIdentifier identifier2 = TableIdentifier.of(namespace, "multiDiffTable2"); + if (requiresNamespaceCreate()) { + catalog().createNamespace(namespace); + } + Table table1 = catalog().buildTable(identifier1, SCHEMA).create(); Table table2 = catalog().buildTable(identifier2, SCHEMA).create(); Transaction t1Transaction = table1.newTransaction(); @@ -2025,6 +2048,10 @@ public void multipleDiffsAgainstMultipleTablesLastFails() { TableIdentifier identifier1 = TableIdentifier.of(namespace, "multiDiffTable1"); TableIdentifier identifier2 = TableIdentifier.of(namespace, "multiDiffTable2"); + if (requiresNamespaceCreate()) { + catalog().createNamespace(namespace); + } + catalog().createTable(identifier1, SCHEMA); catalog().createTable(identifier2, SCHEMA); @@ -2071,7 +2098,12 @@ public void multipleDiffsAgainstMultipleTablesLastFails() { @Test public void testCleanupUncommitedFilesForCleanableFailures() { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); - Catalog catalog = catalog(adapter); + RESTCatalog catalog = catalog(adapter); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + catalog.createTable(TABLE, SCHEMA); DataFile file = DataFiles.builder(PartitionSpec.unpartitioned()) @@ -2102,7 +2134,12 @@ public void testCleanupUncommitedFilesForCleanableFailures() { @Test public void testNoCleanupForNonCleanableExceptions() { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); - Catalog catalog = catalog(adapter); + RESTCatalog catalog = catalog(adapter); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + catalog.createTable(TABLE, SCHEMA); Table table = catalog.loadTable(TABLE); @@ -2127,7 +2164,12 @@ public void testNoCleanupForNonCleanableExceptions() { @Test public void testCleanupCleanableExceptionsCreate() { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); - Catalog catalog = catalog(adapter); + RESTCatalog catalog = catalog(adapter); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + catalog.createTable(TABLE, SCHEMA); TableIdentifier newTable = TableIdentifier.of(TABLE.namespace(), "some_table"); ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateTableRequest.class); @@ -2161,7 +2203,12 @@ public void testCleanupCleanableExceptionsCreate() { @Test public void testNoCleanupForNonCleanableCreateTransaction() { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); - Catalog catalog = catalog(adapter); + RESTCatalog catalog = catalog(adapter); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + catalog.createTable(TABLE, SCHEMA); TableIdentifier newTable = TableIdentifier.of(TABLE.namespace(), "some_table"); Mockito.doThrow(new ServiceFailureException("some service failure")) @@ -2194,7 +2241,12 @@ public void testNoCleanupForNonCleanableCreateTransaction() { @Test public void testCleanupCleanableExceptionsReplace() { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); - Catalog catalog = catalog(adapter); + RESTCatalog catalog = catalog(adapter); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + catalog.createTable(TABLE, SCHEMA); ArgumentCaptor captor = ArgumentCaptor.forClass(UpdateTableRequest.class); Mockito.doThrow(new NotAuthorizedException("not authorized")) @@ -2227,7 +2279,12 @@ public void testCleanupCleanableExceptionsReplace() { @Test public void testNoCleanupForNonCleanableReplaceTransaction() { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); - Catalog catalog = catalog(adapter); + RESTCatalog catalog = catalog(adapter); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(TABLE.namespace()); + } + catalog.createTable(TABLE, SCHEMA); Mockito.doThrow(new ServiceFailureException("some service failure")) .when(adapter) @@ -2256,7 +2313,7 @@ public void testNoCleanupForNonCleanableReplaceTransaction() { .isTrue(); } - private Catalog catalog(RESTCatalogAdapter adapter) { + private RESTCatalog catalog(RESTCatalogAdapter adapter) { RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); catalog.initialize( From 942988744e96e8c133a7fcce267e93942ce556ec Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 9 Nov 2023 13:43:24 -0800 Subject: [PATCH 09/94] Spark 3.5: Fix rewriting manifests for evolved unpartitioned V1 tables (#9015) --- .../actions/RewriteManifestsSparkAction.java | 2 +- .../actions/TestRewriteManifestsAction.java | 63 ++++++++++++++++++- 2 files changed, 61 insertions(+), 4 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index af442ec300bc..0c08324a1a84 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -179,7 +179,7 @@ private RewriteManifests.Result doExecute() { Dataset manifestEntryDF = buildManifestEntryDF(matchingManifests); List newManifests; - if (spec.fields().size() < 1) { + if (spec.isUnpartitioned()) { newManifests = writeManifestsForUnpartitionedTable(manifestEntryDF, targetNumManifests); } else { newManifests = writeManifestsForPartitionedTable(manifestEntryDF, targetNumManifests); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 9ae1954d68bb..d3932c2e82a8 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -44,8 +44,10 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.actions.RewriteManifests; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.hadoop.HadoopTables; @@ -598,6 +600,55 @@ public void testRewriteSmallManifestsNonPartitionedV2Table() { Assert.assertEquals("Rows must match", expectedRecords, actualRecords); } + @Test + public void testRewriteLargeManifestsEvolvedUnpartitionedV1Table() throws IOException { + assumeThat(formatVersion).isEqualTo(1); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + table.updateSpec().removeField("c3").commit(); + + assertThat(table.spec().fields()).hasSize(1).allMatch(field -> field.transform().isVoid()); + + List dataFiles = Lists.newArrayList(); + for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { + dataFiles.add(newDataFile(table, TestHelpers.Row.of(new Object[] {null}))); + } + ManifestFile appendManifest = writeManifest(table, dataFiles); + table.newFastAppend().appendManifest(appendManifest).commit(); + + List originalManifests = table.currentSnapshot().allManifests(table.io()); + ManifestFile originalManifest = Iterables.getOnlyElement(originalManifests); + + // set the target manifest size to a small value to force splitting records into multiple files + table + .updateProperties() + .set( + TableProperties.MANIFEST_TARGET_SIZE_BYTES, + String.valueOf(originalManifest.length() / 2)) + .commit(); + + SparkActions actions = SparkActions.get(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + assertThat(result.rewrittenManifests()).hasSize(1); + assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); + assertManifestsLocation(result.addedManifests()); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).hasSizeGreaterThanOrEqualTo(2); + } + private void writeRecords(List records) { Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); writeDF(df); @@ -657,11 +708,17 @@ private ManifestFile writeManifest(Table table, List files) throws IOE } private DataFile newDataFile(Table table, String partitionPath) { + return newDataFileBuilder(table).withPartitionPath(partitionPath).build(); + } + + private DataFile newDataFile(Table table, StructLike partition) { + return newDataFileBuilder(table).withPartition(partition).build(); + } + + private DataFiles.Builder newDataFileBuilder(Table table) { return DataFiles.builder(table.spec()) .withPath("/path/to/data-" + UUID.randomUUID() + ".parquet") .withFileSizeInBytes(10) - .withPartitionPath(partitionPath) - .withRecordCount(1) - .build(); + .withRecordCount(1); } } From 8c625dd7d21e38235d2864e081c008ef11e0fd20 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 9 Nov 2023 14:54:32 -0800 Subject: [PATCH 10/94] Core: Support replacing delete manifests (#9000) --- .../apache/iceberg/BaseRewriteManifests.java | 13 +- .../org/apache/iceberg/TableTestBase.java | 10 +- .../apache/iceberg/TestRewriteManifests.java | 609 ++++++++++++++++++ 3 files changed, 624 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 1f0d204dc2b1..87768e34894a 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -168,7 +168,7 @@ private ManifestFile copyManifest(ManifestFile manifest) { @Override public List apply(TableMetadata base, Snapshot snapshot) { - List currentManifests = base.currentSnapshot().dataManifests(ops.io()); + List currentManifests = base.currentSnapshot().allManifests(ops.io()); Set currentManifestSet = ImmutableSet.copyOf(currentManifests); validateDeletedManifests(currentManifestSet); @@ -190,7 +190,6 @@ public List apply(TableMetadata base, Snapshot snapshot) { List apply = Lists.newArrayList(); Iterables.addAll(apply, newManifestsWithMetadata); apply.addAll(keptManifests); - apply.addAll(base.currentSnapshot().deleteManifests(ops.io())); return apply; } @@ -242,7 +241,7 @@ private void performRewrite(List currentManifests) { .executeWith(workerPool()) .run( manifest -> { - if (predicate != null && !predicate.test(manifest)) { + if (containsDeletes(manifest) || !matchesPredicate(manifest)) { keptManifests.add(manifest); } else { rewrittenManifests.add(manifest); @@ -268,6 +267,14 @@ private void performRewrite(List currentManifests) { } } + private boolean containsDeletes(ManifestFile manifest) { + return manifest.content() == ManifestContent.DELETES; + } + + private boolean matchesPredicate(ManifestFile manifest) { + return predicate == null || predicate.test(manifest); + } + private void validateDeletedManifests(Set currentManifests) { // directly deleted manifests must be still present in the current snapshot deletedManifests.stream() diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index d50845933eb7..68ce05528964 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -328,19 +328,19 @@ ManifestFile writeManifestWithName(String name, DataFile... files) throws IOExce return writer.toManifestFile(); } - ManifestEntry manifestEntry( - ManifestEntry.Status status, Long snapshotId, DataFile file) { + > ManifestEntry manifestEntry( + ManifestEntry.Status status, Long snapshotId, F file) { return manifestEntry(status, snapshotId, 0L, 0L, file); } - ManifestEntry manifestEntry( + > ManifestEntry manifestEntry( ManifestEntry.Status status, Long snapshotId, Long dataSequenceNumber, Long fileSequenceNumber, - DataFile file) { + F file) { - GenericManifestEntry entry = new GenericManifestEntry<>(table.spec().partitionType()); + GenericManifestEntry entry = new GenericManifestEntry<>(table.spec().partitionType()); switch (status) { case ADDED: if (dataSequenceNumber != null && dataSequenceNumber != 0) { diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index d7daae8b3ed4..8cc7e440686d 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -36,7 +37,9 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -1105,6 +1108,612 @@ public void testRewriteManifestsOnBranchUnsupported() { "Cannot commit to branch someBranch: org.apache.iceberg.BaseRewriteManifests does not support branch commits"); } + @Test + public void testRewriteDataManifestsPreservesDeletes() { + assumeThat(formatVersion).isGreaterThan(1); + + Table table = load(); + + // commit data files + table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + // save the append snapshot info + Snapshot appendSnapshot = table.currentSnapshot(); + long appendSnapshotId = appendSnapshot.snapshotId(); + long appendSnapshotSeq = appendSnapshot.sequenceNumber(); + + // commit delete files + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + + // save the delete snapshot info + Snapshot deleteSnapshot = table.currentSnapshot(); + long deleteSnapshotId = deleteSnapshot.snapshotId(); + long deleteSnapshotSeq = deleteSnapshot.sequenceNumber(); + + // there must be 1 data and 1 delete manifest before the rewrite + assertManifestCounts(table, 1, 1); + + // rewrite manifests and cluster entries by file path + table.rewriteManifests().clusterBy(file -> file.path().toString()).commit(); + + Snapshot rewriteSnapshot = table.currentSnapshot(); + + validateSummary(rewriteSnapshot, 1, 1, 2, 2); + + // the rewrite must replace the original data manifest with 2 new data manifests + List dataManifests = sortedDataManifests(table.io(), rewriteSnapshot); + assertThat(dataManifests).hasSize(2); + validateManifest( + dataManifests.get(0), + dataSeqs(appendSnapshotSeq, appendSnapshotSeq), + fileSeqs(appendSnapshotSeq, appendSnapshotSeq), + ids(appendSnapshotId), + files(FILE_A), + statuses(ManifestEntry.Status.EXISTING)); + validateManifest( + dataManifests.get(1), + dataSeqs(appendSnapshotSeq, appendSnapshotSeq), + fileSeqs(appendSnapshotSeq, appendSnapshotSeq), + ids(appendSnapshotId), + files(FILE_B), + statuses(ManifestEntry.Status.EXISTING)); + + // the rewrite must preserve the original delete manifest (rewriting is not supported yet) + List deleteManifests = rewriteSnapshot.deleteManifests(table.io()); + ManifestFile deleteManifest = Iterables.getOnlyElement(deleteManifests); + validateDeleteManifest( + deleteManifest, + dataSeqs(deleteSnapshotSeq, deleteSnapshotSeq), + fileSeqs(deleteSnapshotSeq, deleteSnapshotSeq), + ids(deleteSnapshotId, deleteSnapshotId), + files(FILE_A_DELETES, FILE_A2_DELETES), + statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED)); + } + + @Test + public void testReplaceDeleteManifestsOnly() throws IOException { + assumeThat(formatVersion).isGreaterThan(1); + + Table table = load(); + + // commit data files + table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + // save the append snapshot info + Snapshot appendSnapshot = table.currentSnapshot(); + long appendSnapshotId = appendSnapshot.snapshotId(); + long appendSnapshotSeq = appendSnapshot.sequenceNumber(); + + // commit delete files + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + + // save the delete snapshot info + Snapshot deleteSnapshot = table.currentSnapshot(); + long deleteSnapshotId = deleteSnapshot.snapshotId(); + long deleteSnapshotSeq = deleteSnapshot.sequenceNumber(); + + // there must be 1 data and 1 delete manifest before the rewrite + assertManifestCounts(table, 1, 1); + + // split the original delete manifest into 2 new delete manifests + ManifestFile originalDeleteManifest = + Iterables.getOnlyElement(deleteSnapshot.deleteManifests(table.io())); + ManifestFile newDeleteManifest1 = + writeManifest( + "delete-manifest-file-1.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId, + deleteSnapshotSeq, + deleteSnapshotSeq, + FILE_A_DELETES)); + ManifestFile newDeleteManifest2 = + writeManifest( + "delete-manifest-file-2.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId, + deleteSnapshotSeq, + deleteSnapshotSeq, + FILE_A2_DELETES)); + + // replace the original delete manifest with the new delete manifests + table + .rewriteManifests() + .deleteManifest(originalDeleteManifest) + .addManifest(newDeleteManifest1) + .addManifest(newDeleteManifest2) + .commit(); + + Snapshot rewriteSnapshot = table.currentSnapshot(); + + // the rewrite must preserve the original data manifest + ManifestFile dataManifest = Iterables.getOnlyElement(rewriteSnapshot.dataManifests(table.io())); + validateManifest( + dataManifest, + dataSeqs(appendSnapshotSeq, appendSnapshotSeq), + fileSeqs(appendSnapshotSeq, appendSnapshotSeq), + ids(appendSnapshotId, appendSnapshotId), + files(FILE_A, FILE_B), + statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED)); + + // the rewrite must replace the original delete manifest with 2 new delete manifests + List deleteManifests = rewriteSnapshot.deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(2); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(deleteSnapshotSeq), + fileSeqs(deleteSnapshotSeq), + ids(deleteSnapshotId), + files(FILE_A_DELETES), + statuses(ManifestEntry.Status.EXISTING)); + validateDeleteManifest( + deleteManifests.get(1), + dataSeqs(deleteSnapshotSeq), + fileSeqs(deleteSnapshotSeq), + ids(deleteSnapshotId), + files(FILE_A2_DELETES), + statuses(ManifestEntry.Status.EXISTING)); + } + + @Test + public void testReplaceDataAndDeleteManifests() throws IOException { + assumeThat(formatVersion).isGreaterThan(1); + + Table table = load(); + + // commit data files + table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + // save the append snapshot info + Snapshot appendSnapshot = table.currentSnapshot(); + long appendSnapshotId = appendSnapshot.snapshotId(); + long appendSnapshotSeq = appendSnapshot.sequenceNumber(); + + // commit delete files + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + + // save the delete snapshot info + Snapshot deleteSnapshot = table.currentSnapshot(); + long deleteSnapshotId = deleteSnapshot.snapshotId(); + long deleteSnapshotSeq = deleteSnapshot.sequenceNumber(); + + // there must be 1 data and 1 delete manifest before the rewrite + assertManifestCounts(table, 1, 1); + + // split the original data manifest into 2 new data manifests + ManifestFile originalDataManifest = + Iterables.getOnlyElement(deleteSnapshot.dataManifests(table.io())); + ManifestFile newDataManifest1 = + writeManifest( + "manifest-file-1.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + appendSnapshotId, + appendSnapshotSeq, + appendSnapshotSeq, + FILE_A)); + ManifestFile newDataManifest2 = + writeManifest( + "manifest-file-2.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + appendSnapshotId, + appendSnapshotSeq, + appendSnapshotSeq, + FILE_B)); + + // split the original delete manifest into 2 new delete manifests + ManifestFile originalDeleteManifest = + Iterables.getOnlyElement(deleteSnapshot.deleteManifests(table.io())); + ManifestFile newDeleteManifest1 = + writeManifest( + "delete-manifest-file-1.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId, + deleteSnapshotSeq, + deleteSnapshotSeq, + FILE_A_DELETES)); + ManifestFile newDeleteManifest2 = + writeManifest( + "delete-manifest-file-2.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId, + deleteSnapshotSeq, + deleteSnapshotSeq, + FILE_A2_DELETES)); + + // replace the original data and delete manifests with new ones + table + .rewriteManifests() + .deleteManifest(originalDataManifest) + .addManifest(newDataManifest1) + .addManifest(newDataManifest2) + .deleteManifest(originalDeleteManifest) + .addManifest(newDeleteManifest1) + .addManifest(newDeleteManifest2) + .commit(); + + Snapshot rewriteSnapshot = table.currentSnapshot(); + + // the rewrite must replace the original data manifest with 2 new data manifests + List dataManifests = sortedDataManifests(table.io(), rewriteSnapshot); + assertThat(dataManifests).hasSize(2); + validateManifest( + dataManifests.get(0), + dataSeqs(appendSnapshotSeq), + fileSeqs(appendSnapshotSeq), + ids(appendSnapshotId), + files(FILE_A), + statuses(ManifestEntry.Status.EXISTING)); + validateManifest( + dataManifests.get(1), + dataSeqs(appendSnapshotSeq), + fileSeqs(appendSnapshotSeq), + ids(appendSnapshotId), + files(FILE_B), + statuses(ManifestEntry.Status.EXISTING)); + + // the rewrite must replace the original delete manifest with 2 new delete manifests + List deleteManifests = rewriteSnapshot.deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(2); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(deleteSnapshotSeq), + fileSeqs(deleteSnapshotSeq), + ids(deleteSnapshotId), + files(FILE_A_DELETES), + statuses(ManifestEntry.Status.EXISTING)); + validateDeleteManifest( + deleteManifests.get(1), + dataSeqs(deleteSnapshotSeq), + fileSeqs(deleteSnapshotSeq), + ids(deleteSnapshotId), + files(FILE_A2_DELETES), + statuses(ManifestEntry.Status.EXISTING)); + } + + @Test + public void testDeleteManifestReplacementConcurrentAppend() throws IOException { + assumeThat(formatVersion).isGreaterThan(1); + + // commit data files + table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + // save the initial append snapshot info + Snapshot appendSnapshot = table.currentSnapshot(); + long appendSnapshotId = appendSnapshot.snapshotId(); + long appendSnapshotSeq = appendSnapshot.sequenceNumber(); + + // commit delete files + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + + // save the delete snapshot info + Snapshot deleteSnapshot = table.currentSnapshot(); + long deleteSnapshotId = deleteSnapshot.snapshotId(); + long deleteSnapshotSeq = deleteSnapshot.sequenceNumber(); + + // split the original delete manifest into 2 new delete manifests + ManifestFile originalDeleteManifest = + Iterables.getOnlyElement(deleteSnapshot.deleteManifests(table.io())); + ManifestFile newDeleteManifest1 = + writeManifest( + "delete-manifest-file-1.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId, + deleteSnapshotSeq, + deleteSnapshotSeq, + FILE_A_DELETES)); + ManifestFile newDeleteManifest2 = + writeManifest( + "delete-manifest-file-2.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId, + deleteSnapshotSeq, + deleteSnapshotSeq, + FILE_A2_DELETES)); + + // start the rewrite + RewriteManifests rewriteManifests = table.rewriteManifests(); + rewriteManifests.deleteManifest(originalDeleteManifest); + rewriteManifests.addManifest(newDeleteManifest1); + rewriteManifests.addManifest(newDeleteManifest2); + + // commit another append concurrently + table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); + + // save the concurrent snapshot info + Snapshot concurrentSnapshot = table.currentSnapshot(); + long concurrentSnapshotSeq = concurrentSnapshot.sequenceNumber(); + long concurrentSnapshotId = concurrentSnapshot.snapshotId(); + + // there must be 2 data manifests and 1 delete manifest before the rewrite is committed + assertManifestCounts(table, 2, 1); + + // commit the rewrite successfully as operations are not in conflict + rewriteManifests.commit(); + + Snapshot rewriteSnapshot = table.currentSnapshot(); + + validateSummary(rewriteSnapshot, 1, 2, 2, 0); + + // the rewrite must preserve the original and added concurrently data manifests + List dataManifests = rewriteSnapshot.dataManifests(table.io()); + assertThat(dataManifests).hasSize(2); + validateManifest( + dataManifests.get(0), + dataSeqs(concurrentSnapshotSeq, concurrentSnapshotSeq), + fileSeqs(concurrentSnapshotSeq, concurrentSnapshotSeq), + ids(concurrentSnapshotId, concurrentSnapshotId), + files(FILE_C, FILE_D), + statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED)); + validateManifest( + dataManifests.get(1), + dataSeqs(appendSnapshotSeq, appendSnapshotSeq), + fileSeqs(appendSnapshotSeq, appendSnapshotSeq), + ids(appendSnapshotId, appendSnapshotId), + files(FILE_A, FILE_B), + statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED)); + + // the rewrite must replace the original delete manifest with 2 new delete manifests + List deleteManifests = rewriteSnapshot.deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(2); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(deleteSnapshotSeq), + fileSeqs(deleteSnapshotSeq), + ids(deleteSnapshotId), + files(FILE_A_DELETES), + statuses(ManifestEntry.Status.EXISTING)); + validateDeleteManifest( + deleteManifests.get(1), + dataSeqs(deleteSnapshotSeq), + fileSeqs(deleteSnapshotSeq), + ids(deleteSnapshotId), + files(FILE_A2_DELETES), + statuses(ManifestEntry.Status.EXISTING)); + } + + @Test + public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IOException { + assumeThat(formatVersion).isGreaterThan(1); + + // commit data files + table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); + + // save the initial append snapshot info + Snapshot appendSnapshot = table.currentSnapshot(); + long appendSnapshotId = appendSnapshot.snapshotId(); + long appendSnapshotSeq = appendSnapshot.sequenceNumber(); + + // commit the first set of delete files + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + + // save the first delete snapshot info + Snapshot deleteSnapshot1 = table.currentSnapshot(); + long deleteSnapshotId1 = deleteSnapshot1.snapshotId(); + long deleteSnapshotSeq1 = deleteSnapshot1.sequenceNumber(); + + // commit the second set of delete files + table.newRowDelta().addDeletes(FILE_B_DELETES).addDeletes(FILE_C2_DELETES).commit(); + + // save the second delete snapshot info + Snapshot deleteSnapshot2 = table.currentSnapshot(); + long deleteSnapshotId2 = deleteSnapshot2.snapshotId(); + long deleteSnapshotSeq2 = deleteSnapshot2.sequenceNumber(); + + // split the original delete manifest into 2 new delete manifests + ManifestFile originalDeleteManifest = deleteSnapshot1.deleteManifests(table.io()).get(0); + ManifestFile newDeleteManifest1 = + writeManifest( + "delete-manifest-file-1.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId1, + deleteSnapshotSeq1, + deleteSnapshotSeq1, + FILE_A_DELETES)); + ManifestFile newDeleteManifest2 = + writeManifest( + "delete-manifest-file-2.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId1, + deleteSnapshotSeq1, + deleteSnapshotSeq1, + FILE_A2_DELETES)); + + // start the rewrite + RewriteManifests rewriteManifests = table.rewriteManifests(); + rewriteManifests.deleteManifest(originalDeleteManifest); + rewriteManifests.addManifest(newDeleteManifest1); + rewriteManifests.addManifest(newDeleteManifest2); + + // commit the third set of delete files concurrently + table.newRewrite().deleteFile(FILE_B_DELETES).commit(); + + Snapshot concurrentSnapshot = table.currentSnapshot(); + long concurrentSnapshotId = concurrentSnapshot.snapshotId(); + + // there must be 1 data manifest and 2 delete manifests before the rewrite is committed + assertManifestCounts(table, 1, 2); + + // commit the rewrite successfully as operations are not in conflict + rewriteManifests.commit(); + + Snapshot rewriteSnapshot = table.currentSnapshot(); + + validateSummary(rewriteSnapshot, 1, 2, 2, 0); + + // the rewrite must preserve the original data manifest + ManifestFile dataManifest = Iterables.getOnlyElement(rewriteSnapshot.dataManifests(table.io())); + validateManifest( + dataManifest, + dataSeqs(appendSnapshotSeq, appendSnapshotSeq), + fileSeqs(appendSnapshotSeq, appendSnapshotSeq), + ids(appendSnapshotId, appendSnapshotId), + files(FILE_A, FILE_B), + statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED)); + + // the rewrite must replace the first delete manifest with 2 new delete manifests + // the rewrite must also keep the second delete manifest modified concurrently + List deleteManifests = rewriteSnapshot.deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(3); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(deleteSnapshotSeq1), + fileSeqs(deleteSnapshotSeq1), + ids(deleteSnapshotId1), + files(FILE_A_DELETES), + statuses(ManifestEntry.Status.EXISTING)); + validateDeleteManifest( + deleteManifests.get(1), + dataSeqs(deleteSnapshotSeq1), + fileSeqs(deleteSnapshotSeq1), + ids(deleteSnapshotId1), + files(FILE_A2_DELETES), + statuses(ManifestEntry.Status.EXISTING)); + validateDeleteManifest( + deleteManifests.get(2), + dataSeqs(deleteSnapshotSeq2, deleteSnapshotSeq2), + fileSeqs(deleteSnapshotSeq2, deleteSnapshotSeq2), + ids(concurrentSnapshotId, deleteSnapshotId2), + files(FILE_B_DELETES, FILE_C2_DELETES), + statuses(ManifestEntry.Status.DELETED, ManifestEntry.Status.EXISTING)); + } + + @Test + public void testDeleteManifestReplacementConflictingDeleteFileRemoval() throws IOException { + assumeThat(formatVersion).isGreaterThan(1); + + // commit data files + table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit(); + + // commit delete files + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); + + // save the delete snapshot info + Snapshot deleteSnapshot = table.currentSnapshot(); + long deleteSnapshotId = deleteSnapshot.snapshotId(); + long deleteSnapshotSeq = deleteSnapshot.sequenceNumber(); + + // split the original delete manifest into 2 new delete manifests + ManifestFile originalDeleteManifest = deleteSnapshot.deleteManifests(table.io()).get(0); + ManifestFile newDeleteManifest1 = + writeManifest( + "delete-manifest-file-1.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId, + deleteSnapshotSeq, + deleteSnapshotSeq, + FILE_A_DELETES)); + ManifestFile newDeleteManifest2 = + writeManifest( + "delete-manifest-file-2.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId, + deleteSnapshotSeq, + deleteSnapshotSeq, + FILE_A2_DELETES)); + + // start the rewrite + RewriteManifests rewriteManifests = table.rewriteManifests(); + rewriteManifests.deleteManifest(originalDeleteManifest); + rewriteManifests.addManifest(newDeleteManifest1); + rewriteManifests.addManifest(newDeleteManifest2); + + // modify the original delete manifest concurrently + table.newRewrite().deleteFile(FILE_A_DELETES).commit(); + + // the rewrite must fail as the original delete manifest was replaced concurrently + Assertions.assertThatThrownBy(rewriteManifests::commit) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Manifest is missing"); + } + + @Test + public void testDeleteManifestReplacementFailure() throws IOException { + assumeThat(formatVersion).isGreaterThan(1); + + // commit a data file + table.newFastAppend().appendFile(FILE_A).commit(); + + // commit the first delete file + table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); + + // save the first delete snapshot info + Snapshot deleteSnapshot1 = table.currentSnapshot(); + long deleteSnapshotId1 = deleteSnapshot1.snapshotId(); + long deleteSnapshotSeq1 = deleteSnapshot1.sequenceNumber(); + + // commit the second delete file + table.newRowDelta().addDeletes(FILE_A2_DELETES).commit(); + + // save the second delete snapshot info + Snapshot deleteSnapshot2 = table.currentSnapshot(); + long deleteSnapshotId2 = deleteSnapshot2.snapshotId(); + long deleteSnapshotSeq2 = deleteSnapshot2.sequenceNumber(); + + // there must be 1 data manifest and 2 delete manifests before the rewrite + assertManifestCounts(table, 1, 2); + + // combine the original delete manifests into 1 new delete manifest + ManifestFile newDeleteManifest = + writeManifest( + "delete-manifest-file.avro", + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId1, + deleteSnapshotSeq1, + deleteSnapshotSeq1, + FILE_A_DELETES), + manifestEntry( + ManifestEntry.Status.EXISTING, + deleteSnapshotId2, + deleteSnapshotSeq2, + deleteSnapshotSeq2, + FILE_A2_DELETES)); + + // configure the table operations to fail + table.updateProperties().set(TableProperties.COMMIT_NUM_RETRIES, "1").commit(); + table.ops().failCommits(5); + + // start the rewrite + RewriteManifests rewriteManifests = table.rewriteManifests(); + List originalDeleteManifests = deleteSnapshot2.deleteManifests(table.io()); + for (ManifestFile originalDeleteManifest : originalDeleteManifests) { + rewriteManifests.deleteManifest(originalDeleteManifest); + } + rewriteManifests.addManifest(newDeleteManifest); + + // the rewrite must fail + Assertions.assertThatThrownBy(rewriteManifests::commit) + .isInstanceOf(CommitFailedException.class) + .hasMessage("Injected failure"); + + // the new manifest must not be deleted as the commit hasn't succeeded + assertThat(new File(newDeleteManifest.path())).exists(); + } + + private void assertManifestCounts( + Table table, int expectedDataManifestCount, int expectedDeleteManifestCount) { + Snapshot snapshot = table.currentSnapshot(); + assertThat(snapshot.dataManifests(table.io())).hasSize(expectedDataManifestCount); + assertThat(snapshot.deleteManifests(table.io())).hasSize(expectedDeleteManifestCount); + } + + private List sortedDataManifests(FileIO io, Snapshot snapshot) { + List manifests = Lists.newArrayList(snapshot.dataManifests(io)); + manifests.sort(Comparator.comparing(ManifestFile::path)); + return manifests; + } + private void validateSummary( Snapshot snapshot, int replaced, int kept, int created, int entryCount) { Map summary = snapshot.summary(); From 255986a8e7c59915ff0f2b98c9021ddbbc45e675 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 9 Nov 2023 15:30:28 -0800 Subject: [PATCH 11/94] Spark 3.4: Use rolling manifest writers when optimizing metadata (#9019) --- .../actions/RewriteManifestsSparkAction.java | 231 +++++++----------- .../actions/TestRewriteManifestsAction.java | 61 +++-- 2 files changed, 134 insertions(+), 158 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index bc2ef2306790..af442ec300bc 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -20,8 +20,7 @@ import static org.apache.iceberg.MetadataTableType.ENTRIES; -import java.io.IOException; -import java.util.Collections; +import java.io.Serializable; import java.util.List; import java.util.UUID; import java.util.function.Function; @@ -36,6 +35,7 @@ import org.apache.iceberg.ManifestWriter; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Partitioning; +import org.apache.iceberg.RollingManifestWriter; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; @@ -167,20 +167,7 @@ private RewriteManifests.Result doExecute() { .build(); } - long totalSizeBytes = 0L; - int numEntries = 0; - - for (ManifestFile manifest : matchingManifests) { - ValidationException.check( - hasFileCounts(manifest), "No file counts in manifest: %s", manifest.path()); - - totalSizeBytes += manifest.length(); - numEntries += - manifest.addedFilesCount() + manifest.existingFilesCount() + manifest.deletedFilesCount(); - } - - int targetNumManifests = targetNumManifests(totalSizeBytes); - int targetNumManifestEntries = targetNumManifestEntries(numEntries, targetNumManifests); + int targetNumManifests = targetNumManifests(totalSizeBytes(matchingManifests)); if (targetNumManifests == 1 && matchingManifests.size() == 1) { return ImmutableRewriteManifests.Result.builder() @@ -195,9 +182,7 @@ private RewriteManifests.Result doExecute() { if (spec.fields().size() < 1) { newManifests = writeManifestsForUnpartitionedTable(manifestEntryDF, targetNumManifests); } else { - newManifests = - writeManifestsForPartitionedTable( - manifestEntryDF, targetNumManifests, targetNumManifestEntries); + newManifests = writeManifestsForPartitionedTable(manifestEntryDF, targetNumManifests); } replaceManifests(matchingManifests, newManifests); @@ -233,41 +218,24 @@ private Dataset buildManifestEntryDF(List manifests) { private List writeManifestsForUnpartitionedTable( Dataset manifestEntryDF, int numManifests) { - Broadcast tableBroadcast = - sparkContext().broadcast(SerializableTableWithSize.copyOf(table)); StructType sparkType = (StructType) manifestEntryDF.schema().apply("data_file").dataType(); Types.StructType combinedPartitionType = Partitioning.partitionType(table); - - // we rely only on the target number of manifests for unpartitioned tables - // as we should not worry about having too much metadata per partition - long maxNumManifestEntries = Long.MAX_VALUE; + Types.StructType partitionType = spec.partitionType(); return manifestEntryDF .repartition(numManifests) .mapPartitions( - toManifests( - tableBroadcast, - maxNumManifestEntries, - outputLocation, - formatVersion, - combinedPartitionType, - spec, - sparkType), + toManifests(manifestWriters(), combinedPartitionType, partitionType, sparkType), manifestEncoder) .collectAsList(); } private List writeManifestsForPartitionedTable( - Dataset manifestEntryDF, int numManifests, int targetNumManifestEntries) { + Dataset manifestEntryDF, int numManifests) { - Broadcast
tableBroadcast = - sparkContext().broadcast(SerializableTableWithSize.copyOf(table)); StructType sparkType = (StructType) manifestEntryDF.schema().apply("data_file").dataType(); Types.StructType combinedPartitionType = Partitioning.partitionType(table); - - // we allow the actual size of manifests to be 10% higher if the estimation is not precise - // enough - long maxNumManifestEntries = (long) (1.1 * targetNumManifestEntries); + Types.StructType partitionType = spec.partitionType(); return withReusableDS( manifestEntryDF, @@ -276,14 +244,7 @@ private List writeManifestsForPartitionedTable( return df.repartitionByRange(numManifests, partitionColumn) .sortWithinPartitions(partitionColumn) .mapPartitions( - toManifests( - tableBroadcast, - maxNumManifestEntries, - outputLocation, - formatVersion, - combinedPartitionType, - spec, - sparkType), + toManifests(manifestWriters(), combinedPartitionType, partitionType, sparkType), manifestEncoder) .collectAsList(); }); @@ -319,8 +280,16 @@ private int targetNumManifests(long totalSizeBytes) { return (int) ((totalSizeBytes + targetManifestSizeBytes - 1) / targetManifestSizeBytes); } - private int targetNumManifestEntries(int numEntries, int numManifests) { - return (numEntries + numManifests - 1) / numManifests; + private long totalSizeBytes(Iterable manifests) { + long totalSizeBytes = 0L; + + for (ManifestFile manifest : manifests) { + ValidationException.check( + hasFileCounts(manifest), "No file counts in manifest: %s", manifest.path()); + totalSizeBytes += manifest.length(); + } + + return totalSizeBytes; } private boolean hasFileCounts(ManifestFile manifest) { @@ -360,104 +329,90 @@ private void deleteFiles(Iterable locations) { .run(location -> table.io().deleteFile(location)); } - private static ManifestFile writeManifest( - List rows, - int startIndex, - int endIndex, - Broadcast
tableBroadcast, - String location, - int format, - Types.StructType combinedPartitionType, - PartitionSpec spec, - StructType sparkType) - throws IOException { + private ManifestWriterFactory manifestWriters() { + return new ManifestWriterFactory( + sparkContext().broadcast(SerializableTableWithSize.copyOf(table)), + formatVersion, + spec.specId(), + outputLocation, + // allow the actual size of manifests to be 20% higher as the estimation is not precise + (long) (1.2 * targetManifestSizeBytes)); + } - String manifestName = "optimized-m-" + UUID.randomUUID(); - Path manifestPath = new Path(location, manifestName); - OutputFile outputFile = - tableBroadcast - .value() - .io() - .newOutputFile(FileFormat.AVRO.addExtension(manifestPath.toString())); + private static MapPartitionsFunction toManifests( + ManifestWriterFactory writers, + Types.StructType combinedPartitionType, + Types.StructType partitionType, + StructType sparkType) { - Types.StructType combinedFileType = DataFile.getType(combinedPartitionType); - Types.StructType manifestFileType = DataFile.getType(spec.partitionType()); - SparkDataFile wrapper = new SparkDataFile(combinedFileType, manifestFileType, sparkType); + return rows -> { + Types.StructType combinedFileType = DataFile.getType(combinedPartitionType); + Types.StructType manifestFileType = DataFile.getType(partitionType); + SparkDataFile wrapper = new SparkDataFile(combinedFileType, manifestFileType, sparkType); + + RollingManifestWriter writer = writers.newRollingManifestWriter(); + + try { + while (rows.hasNext()) { + Row row = rows.next(); + long snapshotId = row.getLong(0); + long sequenceNumber = row.getLong(1); + Long fileSequenceNumber = row.isNullAt(2) ? null : row.getLong(2); + Row file = row.getStruct(3); + writer.existing(wrapper.wrap(file), snapshotId, sequenceNumber, fileSequenceNumber); + } + } finally { + writer.close(); + } - ManifestWriter writer = ManifestFiles.write(format, spec, outputFile, null); + return writer.toManifestFiles().iterator(); + }; + } - try { - for (int index = startIndex; index < endIndex; index++) { - Row row = rows.get(index); - long snapshotId = row.getLong(0); - long sequenceNumber = row.getLong(1); - Long fileSequenceNumber = row.isNullAt(2) ? null : row.getLong(2); - Row file = row.getStruct(3); - writer.existing(wrapper.wrap(file), snapshotId, sequenceNumber, fileSequenceNumber); - } - } finally { - writer.close(); + private static class ManifestWriterFactory implements Serializable { + private final Broadcast
tableBroadcast; + private final int formatVersion; + private final int specId; + private final String outputLocation; + private final long maxManifestSizeBytes; + + ManifestWriterFactory( + Broadcast
tableBroadcast, + int formatVersion, + int specId, + String outputLocation, + long maxManifestSizeBytes) { + this.tableBroadcast = tableBroadcast; + this.formatVersion = formatVersion; + this.specId = specId; + this.outputLocation = outputLocation; + this.maxManifestSizeBytes = maxManifestSizeBytes; } - return writer.toManifestFile(); - } + public RollingManifestWriter newRollingManifestWriter() { + return new RollingManifestWriter<>(this::newManifestWriter, maxManifestSizeBytes); + } - private static MapPartitionsFunction toManifests( - Broadcast
tableBroadcast, - long maxNumManifestEntries, - String location, - int format, - Types.StructType combinedPartitionType, - PartitionSpec spec, - StructType sparkType) { + private ManifestWriter newManifestWriter() { + return ManifestFiles.write(formatVersion, spec(), newOutputFile(), null); + } - return rows -> { - List rowsAsList = Lists.newArrayList(rows); + private PartitionSpec spec() { + return table().specs().get(specId); + } - if (rowsAsList.isEmpty()) { - return Collections.emptyIterator(); - } + private OutputFile newOutputFile() { + return table().io().newOutputFile(newManifestLocation()); + } - List manifests = Lists.newArrayList(); - if (rowsAsList.size() <= maxNumManifestEntries) { - manifests.add( - writeManifest( - rowsAsList, - 0, - rowsAsList.size(), - tableBroadcast, - location, - format, - combinedPartitionType, - spec, - sparkType)); - } else { - int midIndex = rowsAsList.size() / 2; - manifests.add( - writeManifest( - rowsAsList, - 0, - midIndex, - tableBroadcast, - location, - format, - combinedPartitionType, - spec, - sparkType)); - manifests.add( - writeManifest( - rowsAsList, - midIndex, - rowsAsList.size(), - tableBroadcast, - location, - format, - combinedPartitionType, - spec, - sparkType)); - } + private String newManifestLocation() { + String fileName = FileFormat.AVRO.addExtension("optimized-m-" + UUID.randomUUID()); + Path filePath = new Path(outputLocation, fileName); + return filePath.toString(); + } - return manifests.iterator(); - }; + private Table table() { + return tableBroadcast.value(); + } } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 4ce5ba4e9d6a..9ae1954d68bb 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -34,9 +34,13 @@ import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -45,6 +49,7 @@ import org.apache.iceberg.actions.RewriteManifests; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -412,16 +417,12 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); Table table = TABLES.create(SCHEMA, spec, options, tableLocation); - // all records belong to the same partition - List records = Lists.newArrayList(); - for (int i = 0; i < 50; i++) { - records.add(new ThreeColumnRecord(i, String.valueOf(i), "0")); + List dataFiles = Lists.newArrayList(); + for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { + dataFiles.add(newDataFile(table, "c3=" + fileOrdinal)); } - Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); - // repartition to create separate files - writeDF(df.repartition(50, df.col("c1"))); - - table.refresh(); + ManifestFile appendManifest = writeManifest(table, dataFiles); + table.newFastAppend().appendManifest(appendManifest).commit(); List manifests = table.currentSnapshot().allManifests(table.io()); Assert.assertEquals("Should have 1 manifests before rewrite", 1, manifests.size()); @@ -446,22 +447,14 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { .stagingLocation(stagingLocation) .execute(); - Assert.assertEquals( - "Action should rewrite 1 manifest", 1, Iterables.size(result.rewrittenManifests())); - Assert.assertEquals( - "Action should add 2 manifests", 2, Iterables.size(result.addedManifests())); + Assertions.assertThat(result.rewrittenManifests()).hasSize(1); + Assertions.assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); assertManifestsLocation(result.addedManifests(), stagingLocation); table.refresh(); List newManifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 2 manifests after rewrite", 2, newManifests.size()); - - Dataset resultDF = spark.read().format("iceberg").load(tableLocation); - List actualRecords = - resultDF.sort("c1", "c2").as(Encoders.bean(ThreeColumnRecord.class)).collectAsList(); - - Assert.assertEquals("Rows must match", records, actualRecords); + Assertions.assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); } @Test @@ -643,4 +636,32 @@ private void assertManifestsLocation(Iterable manifests, String st assertThat(manifests).allMatch(manifest -> manifest.path().startsWith(tableLocation)); } } + + private ManifestFile writeManifest(Table table, List files) throws IOException { + File manifestFile = temp.newFile("generated-manifest.avro"); + Assert.assertTrue(manifestFile.delete()); + OutputFile outputFile = table.io().newOutputFile(manifestFile.getCanonicalPath()); + + ManifestWriter writer = + ManifestFiles.write(formatVersion, table.spec(), outputFile, null); + + try { + for (DataFile file : files) { + writer.add(file); + } + } finally { + writer.close(); + } + + return writer.toManifestFile(); + } + + private DataFile newDataFile(Table table, String partitionPath) { + return DataFiles.builder(table.spec()) + .withPath("/path/to/data-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(10) + .withPartitionPath(partitionPath) + .withRecordCount(1) + .build(); + } } From 7cec1d97965a2823878aef17d5c54fc768e8612f Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 9 Nov 2023 16:26:11 -0800 Subject: [PATCH 12/94] Docs: Fix Javadoc for ManifestFile (#9016) --- .../java/org/apache/iceberg/ManifestFile.java | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/ManifestFile.java b/api/src/main/java/org/apache/iceberg/ManifestFile.java index ee38a6d4140a..8f20697c7780 100644 --- a/api/src/main/java/org/apache/iceberg/ManifestFile.java +++ b/api/src/main/java/org/apache/iceberg/ManifestFile.java @@ -25,7 +25,7 @@ import java.util.List; import org.apache.iceberg.types.Types; -/** Represents a manifest file that can be scanned to find data files in a table. */ +/** Represents a manifest file that can be scanned to find files in a table. */ public interface ManifestFile { Types.NestedField PATH = required(500, "manifest_path", Types.StringType.get(), "Location URI with FS scheme"); @@ -141,10 +141,10 @@ default boolean hasAddedFiles() { return addedFilesCount() == null || addedFilesCount() > 0; } - /** Returns the number of data files with status ADDED in the manifest file. */ + /** Returns the number of files with status ADDED in the manifest file. */ Integer addedFilesCount(); - /** Returns the total number of rows in all data files with status ADDED in the manifest file. */ + /** Returns the total number of rows in all files with status ADDED in the manifest file. */ Long addedRowsCount(); /** @@ -156,12 +156,10 @@ default boolean hasExistingFiles() { return existingFilesCount() == null || existingFilesCount() > 0; } - /** Returns the number of data files with status EXISTING in the manifest file. */ + /** Returns the number of files with status EXISTING in the manifest file. */ Integer existingFilesCount(); - /** - * Returns the total number of rows in all data files with status EXISTING in the manifest file. - */ + /** Returns the total number of rows in all files with status EXISTING in the manifest file. */ Long existingRowsCount(); /** @@ -173,12 +171,10 @@ default boolean hasDeletedFiles() { return deletedFilesCount() == null || deletedFilesCount() > 0; } - /** Returns the number of data files with status DELETED in the manifest file. */ + /** Returns the number of files with status DELETED in the manifest file. */ Integer deletedFilesCount(); - /** - * Returns the total number of rows in all data files with status DELETED in the manifest file. - */ + /** Returns the total number of rows in all files with status DELETED in the manifest file. */ Long deletedRowsCount(); /** @@ -214,12 +210,12 @@ static Types.StructType getType() { return PARTITION_SUMMARY_TYPE; } - /** Returns true if at least one data file in the manifest has a null value for the field. */ + /** Returns true if at least one file in the manifest has a null value for the field. */ boolean containsNull(); /** - * Returns true if at least one data file in the manifest has a NaN value for the field. Null if - * this information doesn't exist. + * Returns true if at least one file in the manifest has a NaN value for the field. Null if this + * information doesn't exist. * *

Default to return null to ensure backward compatibility. */ From 09e6a9f7b8d733ca7a02c3933b8c88917bdb8477 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Fri, 10 Nov 2023 17:01:58 +0530 Subject: [PATCH 13/94] Spec: Fix view example (#8966) --- format/view-spec.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/format/view-spec.md b/format/view-spec.md index d6064787bfe3..4ba7c6cbc9de 100644 --- a/format/view-spec.md +++ b/format/view-spec.md @@ -239,12 +239,14 @@ s3://bucket/warehouse/default.db/event_agg/metadata/00001-(uuid).metadata.json ``` Each change creates a new metadata JSON file. +In the below example, the underlying SQL is modified by specifying the fully-qualified table name. ```sql USE prod.other_db; CREATE OR REPLACE VIEW default.event_agg ( - event_count, + event_count COMMENT 'Count of events', event_date) +COMMENT 'Daily event counts' AS SELECT COUNT(1), CAST(event_ts AS DATE) From 774d0e8bc9528b057201d8e2cea31405bd49afbd Mon Sep 17 00:00:00 2001 From: zhaoym Date: Sat, 11 Nov 2023 02:51:28 +0800 Subject: [PATCH 14/94] Docs: `DataFrameReader` does not take parameters (#9021) --- docs/spark-queries.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/spark-queries.md b/docs/spark-queries.md index 6e6637e229d4..54b1ee6bd120 100644 --- a/docs/spark-queries.md +++ b/docs/spark-queries.md @@ -176,7 +176,7 @@ To read appended data incrementally, use: ```scala // get the data added after start-snapshot-id (10963874102873L) until end-snapshot-id (63874143573109L) -spark.read() +spark.read .format("iceberg") .option("start-snapshot-id", "10963874102873") .option("end-snapshot-id", "63874143573109") From 6a9c182b47476025da1cfb741cc2f581e955330b Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 10 Nov 2023 11:36:01 -0800 Subject: [PATCH 15/94] Spark 3.5: Set useCommitCoordinator to false in batch writes (#9017) Co-authored-by: Huaxin Gao --- .../iceberg/spark/source/SparkPositionDeletesRewrite.java | 5 +++++ .../apache/iceberg/spark/source/SparkPositionDeltaWrite.java | 5 +++++ .../java/org/apache/iceberg/spark/source/SparkWrite.java | 5 +++++ 3 files changed, 15 insertions(+) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java index d0769eaa5f4e..a397a069ee1d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java @@ -136,6 +136,11 @@ public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { writeProperties); } + @Override + public boolean useCommitCoordinator() { + return false; + } + @Override public void commit(WriterCommitMessage[] messages) { PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index 6c0fc591ffbd..dcc949b290af 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -171,6 +171,11 @@ public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { return new PositionDeltaWriteFactory(tableBroadcast, command, context, writeProperties); } + @Override + public boolean useCommitCoordinator() { + return false; + } + @Override public void commit(WriterCommitMessage[] messages) { RowDelta rowDelta = table.newRowDelta(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index 97359d0a3ae9..a7c1234a79a3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -271,6 +271,11 @@ public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { return createWriterFactory(); } + @Override + public boolean useCommitCoordinator() { + return false; + } + @Override public void abort(WriterCommitMessage[] messages) { SparkWrite.this.abort(messages); From 13fd06d90ddc6f5c6bacf79b2c0dbe64838e7b65 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 11 Nov 2023 10:15:38 -0800 Subject: [PATCH 16/94] Spark 3.5: Set useCommitCoordinator to false in streaming writes (#9027) Co-authored-by: Huaxin Gao --- .../java/org/apache/iceberg/spark/source/SparkWrite.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index a7c1234a79a3..58d77695f27d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -506,6 +506,11 @@ public StreamingDataWriterFactory createStreamingWriterFactory(PhysicalWriteInfo return createWriterFactory(); } + @Override + public boolean useCommitCoordinator() { + return false; + } + @Override public final void commit(long epochId, WriterCommitMessage[] messages) { LOG.info("Committing epoch {} for query {} in {} mode", epochId, queryId, mode()); From d7f8e91c0983048768be1cb17893a485088da937 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 11 Nov 2023 10:17:24 -0800 Subject: [PATCH 17/94] Spark 3.4: Set useCommitCoordinator to false in batch writes (#9028) This change cherrypicks PR #9017 to Spark 3.4. Co-authored-by: Huaxin Gao --- .../iceberg/spark/source/SparkPositionDeletesRewrite.java | 5 +++++ .../apache/iceberg/spark/source/SparkPositionDeltaWrite.java | 5 +++++ .../java/org/apache/iceberg/spark/source/SparkWrite.java | 5 +++++ 3 files changed, 15 insertions(+) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java index d0769eaa5f4e..a397a069ee1d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java @@ -136,6 +136,11 @@ public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { writeProperties); } + @Override + public boolean useCommitCoordinator() { + return false; + } + @Override public void commit(WriterCommitMessage[] messages) { PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index 9fea33948b3e..791cbd8462a4 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -160,6 +160,11 @@ public DeltaWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { return new PositionDeltaWriteFactory(tableBroadcast, command, context, writeProperties); } + @Override + public boolean useCommitCoordinator() { + return false; + } + @Override public void commit(WriterCommitMessage[] messages) { RowDelta rowDelta = table.newRowDelta(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index 15881098e7a3..96432178247d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -260,6 +260,11 @@ public DataWriterFactory createBatchWriterFactory(PhysicalWriteInfo info) { return createWriterFactory(); } + @Override + public boolean useCommitCoordinator() { + return false; + } + @Override public void abort(WriterCommitMessage[] messages) { SparkWrite.this.abort(messages); From fd4231f5e27c6a98bbf4ff38850e1700b7584c0a Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Sun, 12 Nov 2023 19:09:55 -0800 Subject: [PATCH 18/94] API: Optimize equals in CharSequenceWrapper (#9035) --- .../org/apache/iceberg/util/CharSequenceWrapper.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java b/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java index 28c44497012b..e0221206024a 100644 --- a/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java +++ b/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java @@ -52,6 +52,15 @@ public boolean equals(Object other) { } CharSequenceWrapper that = (CharSequenceWrapper) other; + + if (wrapped instanceof String && that.wrapped instanceof String) { + return wrapped.equals(that.wrapped); + } + + if (length() != that.length()) { + return false; + } + return Comparators.charSequences().compare(wrapped, that.wrapped) == 0; } From 930750d3e21f38eb7a5ca53438279ad2a10656cf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Nov 2023 09:59:56 +0100 Subject: [PATCH 19/94] Build: Bump software.amazon.awssdk:bom from 2.21.15 to 2.21.21 (#9044) Bumps software.amazon.awssdk:bom from 2.21.15 to 2.21.21. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index a09b30c56302..77fb30eda952 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -27,7 +27,7 @@ arrow = "14.0.0" avro = "1.11.3" assertj-core = "3.24.2" awaitility = "4.2.0" -awssdk-bom = "2.21.15" +awssdk-bom = "2.21.21" azuresdk-bom = "1.2.18" caffeine = "2.9.3" calcite = "1.10.0" From 7ece5faaf839dfe8adb113b8b2fb1e94245f01cf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Nov 2023 10:00:17 +0100 Subject: [PATCH 20/94] Build: Bump orc from 1.9.1 to 1.9.2 (#9045) Bumps `orc` from 1.9.1 to 1.9.2. Updates `org.apache.orc:orc-core` from 1.9.1 to 1.9.2 Updates `org.apache.orc:orc-tools` from 1.9.1 to 1.9.2 --- updated-dependencies: - dependency-name: org.apache.orc:orc-core dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.orc:orc-tools dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 77fb30eda952..87ef5c3ab36f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -68,7 +68,7 @@ nessie = "0.73.0" netty-buffer = "4.1.97.Final" netty-buffer-compat = "4.1.68.Final" object-client-bundle = "3.3.2" -orc = "1.9.1" +orc = "1.9.2" parquet = "1.13.1" pig = "0.17.0" roaringbitmap = "1.0.0" From 28a95652540515252728b90e85d00e09f5d67f31 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Nov 2023 10:00:35 +0100 Subject: [PATCH 21/94] Build: Bump arrow from 14.0.0 to 14.0.1 (#9043) Bumps `arrow` from 14.0.0 to 14.0.1. Updates `org.apache.arrow:arrow-memory-netty` from 14.0.0 to 14.0.1 Updates `org.apache.arrow:arrow-vector` from 14.0.0 to 14.0.1 - [Commits](https://github.com/apache/arrow/compare/go/v14.0.0...go/v14.0.1) --- updated-dependencies: - dependency-name: org.apache.arrow:arrow-memory-netty dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.arrow:arrow-vector dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 87ef5c3ab36f..e934320feb19 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -23,7 +23,7 @@ activation = "1.1.1" aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" aircompressor = "0.25" -arrow = "14.0.0" +arrow = "14.0.1" avro = "1.11.3" assertj-core = "3.24.2" awaitility = "4.2.0" From 7f21b538f30809e77fef32fce2138d0322fa73c8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Nov 2023 10:00:52 +0100 Subject: [PATCH 22/94] Build: Bump net.snowflake:snowflake-jdbc from 3.14.2 to 3.14.3 (#9039) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.14.2 to 3.14.3. - [Release notes](https://github.com/snowflakedb/snowflake-jdbc/releases) - [Changelog](https://github.com/snowflakedb/snowflake-jdbc/blob/master/CHANGELOG.rst) - [Commits](https://github.com/snowflakedb/snowflake-jdbc/compare/v3.14.2...v3.14.3) --- updated-dependencies: - dependency-name: net.snowflake:snowflake-jdbc dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index e934320feb19..a1cc509f6c51 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -75,7 +75,7 @@ roaringbitmap = "1.0.0" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" -snowflake-jdbc = "3.14.2" +snowflake-jdbc = "3.14.3" spark-hive32 = "3.2.2" spark-hive33 = "3.3.2" spark-hive34 = "3.4.1" From 9476f62ff2ea1220df85376c906a46596008d42b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Nov 2023 10:09:10 +0100 Subject: [PATCH 23/94] Build: Bump junit from 5.10.0 to 5.10.1 (#9037) Bumps `junit` from 5.10.0 to 5.10.1. Updates `org.junit.jupiter:junit-jupiter` from 5.10.0 to 5.10.1 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/compare/r5.10.0...r5.10.1) Updates `org.junit.jupiter:junit-jupiter-engine` from 5.10.0 to 5.10.1 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/compare/r5.10.0...r5.10.1) Updates `org.junit.vintage:junit-vintage-engine` from 5.10.0 to 5.10.1 - [Release notes](https://github.com/junit-team/junit5/releases) - [Commits](https://github.com/junit-team/junit5/compare/r5.10.0...r5.10.1) --- updated-dependencies: - dependency-name: org.junit.jupiter:junit-jupiter dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.jupiter:junit-jupiter-engine dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.junit.vintage:junit-vintage-engine dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index a1cc509f6c51..62a30ca9df83 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -59,7 +59,7 @@ jakarta-el-api = "3.0.3" jaxb-api = "2.3.1" jaxb-runtime = "2.3.3" jetty = "9.4.53.v20231009" -junit = "5.10.0" +junit = "5.10.1" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" From 7ff8f264581c0878e85c360b92a79f925aad758e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Nov 2023 10:09:36 +0100 Subject: [PATCH 24/94] Build: Bump com.google.cloud:libraries-bom from 26.26.0 to 26.27.0 (#9036) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.26.0 to 26.27.0. - [Release notes](https://github.com/googleapis/java-cloud-bom/releases) - [Changelog](https://github.com/googleapis/java-cloud-bom/blob/main/release-please-config.json) - [Commits](https://github.com/googleapis/java-cloud-bom/compare/v26.26.0...v26.27.0) --- updated-dependencies: - dependency-name: com.google.cloud:libraries-bom dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 62a30ca9df83..fb81c2d6cd89 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -39,7 +39,7 @@ findbugs-jsr305 = "3.0.2" flink115 = { strictly = "[1.15, 1.16[", prefer = "1.15.0"} # see rich version usage explanation above flink116 = { strictly = "[1.16, 1.17[", prefer = "1.16.2"} flink117 = { strictly = "[1.17, 1.18[", prefer = "1.17.1"} -google-libraries-bom = "26.26.0" +google-libraries-bom = "26.27.0" guava = "32.1.3-jre" hadoop2 = "2.7.3" hadoop3-client = "3.3.6" From fd00207cdddf179828e3fde788d3f68933fea061 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 Nov 2023 14:26:25 +0100 Subject: [PATCH 25/94] Build: Bump org.xerial:sqlite-jdbc from 3.43.2.1 to 3.44.0.0 (#9051) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.43.2.1 to 3.44.0.0. - [Release notes](https://github.com/xerial/sqlite-jdbc/releases) - [Changelog](https://github.com/xerial/sqlite-jdbc/blob/master/CHANGELOG) - [Commits](https://github.com/xerial/sqlite-jdbc/compare/3.43.2.1...3.44.0.0) --- updated-dependencies: - dependency-name: org.xerial:sqlite-jdbc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index fb81c2d6cd89..e579dae8e99f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -82,7 +82,7 @@ spark-hive34 = "3.4.1" spark-hive35 = "3.5.0" spring-boot = "2.5.4" spring-web = "5.3.30" -sqlite-jdbc = "3.43.2.1" +sqlite-jdbc = "3.44.0.0" testcontainers = "1.19.1" tez010 = "0.10.2" tez08 = { strictly = "[0.8, 0.9[", prefer = "0.8.4"} # see rich version usage explanation above From efa5945f12e85076bd665e5fa2bef53f4cf3f945 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 13 Nov 2023 23:58:05 +0100 Subject: [PATCH 26/94] Add dependabot to automatically update the site (#9004) Since we're moving the site back into the main repository, it would be great to have Dependabot update `site/requirements.txt` --- .github/dependabot.yml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index ad448a30117f..86d4e469ea5b 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -30,4 +30,10 @@ updates: interval: "weekly" day: "sunday" open-pull-requests-limit: 50 + - package-ecosystem: "pip" + directory: "/" + schedule: + interval: "weekly" + day: "sunday" + open-pull-requests-limit: 5 From 9cfbbdc9ef8a39bfc800a1cb7892b3900bd57be7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 14 Nov 2023 07:58:28 +0100 Subject: [PATCH 27/94] Build: Bump mkdocs-macros-plugin from 1.0.4 to 1.0.5 (#9058) Bumps [mkdocs-macros-plugin](https://github.com/fralau/mkdocs_macros_plugin) from 1.0.4 to 1.0.5. - [Release notes](https://github.com/fralau/mkdocs_macros_plugin/releases) - [Changelog](https://github.com/fralau/mkdocs_macros_plugin/blob/master/CHANGELOG.md) - [Commits](https://github.com/fralau/mkdocs_macros_plugin/compare/v1.0.4...v1.0.5) --- updated-dependencies: - dependency-name: mkdocs-macros-plugin dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 58e6d4982f00..d50e4be384ce 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -16,7 +16,7 @@ # under the License. mkdocs-awesome-pages-plugin==2.9.2 -mkdocs-macros-plugin==1.0.4 +mkdocs-macros-plugin==1.0.5 mkdocs-material==9.1.21 mkdocs-material-extensions==1.1.1 mkdocs-monorepo-plugin==1.0.5 From 9c51427f18f37d94aaa55913de0c4b0bb1195608 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 14 Nov 2023 07:59:04 +0100 Subject: [PATCH 28/94] Build: Bump datamodel-code-generator from 0.22.0 to 0.23.0 (#9054) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.22.0 to 0.23.0. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.22.0...0.23.0) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index e4d9181f1c2a..6fa2df8d82c5 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,6 +16,6 @@ # under the License. openapi-spec-validator==0.5.2 -datamodel-code-generator==0.22.0 +datamodel-code-generator==0.23.0 # Add the Pydantic constraint since 2.4.0 has a bug pydantic<2.4.0 From 75ff4a99a9f72bf8735f6062d671ed088380e94e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 14 Nov 2023 08:09:51 +0100 Subject: [PATCH 29/94] Build: Bump mkdocs-material-extensions from 1.1.1 to 1.3 (#9052) Bumps [mkdocs-material-extensions](https://github.com/facelessuser/mkdocs-material-extensions) from 1.1.1 to 1.3. - [Release notes](https://github.com/facelessuser/mkdocs-material-extensions/releases) - [Changelog](https://github.com/facelessuser/mkdocs-material-extensions/blob/master/changelog.md) - [Commits](https://github.com/facelessuser/mkdocs-material-extensions/compare/1.1.1...1.3) --- updated-dependencies: - dependency-name: mkdocs-material-extensions dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index d50e4be384ce..078623f392a3 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -18,6 +18,6 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 mkdocs-material==9.1.21 -mkdocs-material-extensions==1.1.1 +mkdocs-material-extensions==1.3 mkdocs-monorepo-plugin==1.0.5 mkdocs-redirects==1.2.1 From 69025712b41e744304e2a7d971ccdad784e25bec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 14 Nov 2023 09:47:44 +0100 Subject: [PATCH 30/94] Build: Bump mkdocs-material from 9.1.21 to 9.4.8 (#9055) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.1.21 to 9.4.8. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.1.21...9.4.8) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 078623f392a3..a3fc59f97d44 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.1.21 +mkdocs-material==9.4.8 mkdocs-material-extensions==1.3 mkdocs-monorepo-plugin==1.0.5 mkdocs-redirects==1.2.1 From 188847428309701c2ec0debf9e94da94b39058b6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 14 Nov 2023 10:36:30 +0100 Subject: [PATCH 31/94] Build: Bump software.amazon.awssdk:bom from 2.21.21 to 2.21.22 (#9053) Bumps software.amazon.awssdk:bom from 2.21.21 to 2.21.22. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index e579dae8e99f..aff898260ac0 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -27,7 +27,7 @@ arrow = "14.0.1" avro = "1.11.3" assertj-core = "3.24.2" awaitility = "4.2.0" -awssdk-bom = "2.21.21" +awssdk-bom = "2.21.22" azuresdk-bom = "1.2.18" caffeine = "2.9.3" calcite = "1.10.0" From fba7d612539fe978e0281e89bb351e821fd02a05 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 14 Nov 2023 18:58:25 +0100 Subject: [PATCH 32/94] GCP: Use correct Guava imports (#9067) --- .../main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java | 2 +- .../main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java index fbb8f764de6f..fea51d50cfd0 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.gcp.gcs; -import com.google.api.client.util.Lists; import com.google.cloud.ReadChannel; import com.google.cloud.storage.BlobId; import com.google.cloud.storage.Storage; @@ -38,6 +37,7 @@ import org.apache.iceberg.metrics.MetricsContext.Unit; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java index 59caf027aae9..bcd6099ac987 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.gcp.gcs; -import com.google.api.client.util.Lists; import com.google.cloud.WriteChannel; import com.google.cloud.storage.BlobId; import com.google.cloud.storage.BlobInfo; @@ -36,6 +35,7 @@ import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.metrics.MetricsContext.Unit; import org.apache.iceberg.relocated.com.google.common.base.Joiner; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 6ec3de390d3fa6e797c6975b1eaaea41719db0fe Mon Sep 17 00:00:00 2001 From: pvary Date: Tue, 14 Nov 2023 22:09:08 +0100 Subject: [PATCH 33/94] Core: Enable column statistics filtering after planning (#8803) --- .palantir/revapi.yml | 5 + .../org/apache/iceberg/BatchScanAdapter.java | 5 + .../java/org/apache/iceberg/ContentFile.java | 15 ++ .../main/java/org/apache/iceberg/Scan.java | 14 ++ .../java/org/apache/iceberg/TestHelpers.java | 6 + .../iceberg/BaseDistributedDataScan.java | 7 +- .../java/org/apache/iceberg/BaseFile.java | 30 ++-- .../iceberg/BaseIncrementalAppendScan.java | 3 +- .../iceberg/BaseIncrementalChangelogScan.java | 3 +- .../java/org/apache/iceberg/BaseScan.java | 18 +++ .../java/org/apache/iceberg/DataScan.java | 3 +- .../org/apache/iceberg/DataTableScan.java | 3 +- .../org/apache/iceberg/GenericDataFile.java | 19 ++- .../org/apache/iceberg/GenericDeleteFile.java | 19 ++- .../iceberg/IncrementalDataTableScan.java | 3 +- .../org/apache/iceberg/ManifestGroup.java | 21 ++- .../org/apache/iceberg/TableScanContext.java | 14 ++ .../java/org/apache/iceberg/V1Metadata.java | 6 + .../java/org/apache/iceberg/V2Metadata.java | 6 + .../apache/iceberg/util/ContentFileUtil.java | 44 ++++++ .../apache/iceberg/util/SerializableMap.java | 17 +++ .../iceberg/TestScanDataFileColumns.java | 56 ++++--- .../flink/source/FlinkSplitPlanner.java | 4 + .../iceberg/flink/source/ScanContext.java | 17 +++ .../TestContinuousSplitPlannerImpl.java | 143 ++++++++++++++++-- 25 files changed, 422 insertions(+), 59 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 3392c4ae040e..283697418b6b 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -872,6 +872,11 @@ acceptedBreaks: old: "method void org.apache.iceberg.encryption.Ciphers::()" new: "method void org.apache.iceberg.encryption.Ciphers::()" justification: "Static utility class - should not have public constructor" + "1.4.0": + org.apache.iceberg:iceberg-core: + - code: "java.field.serialVersionUIDChanged" + new: "field org.apache.iceberg.util.SerializableMap.serialVersionUID" + justification: "Serialization is not be used" apache-iceberg-0.14.0: org.apache.iceberg:iceberg-api: - code: "java.class.defaultSerializationChanged" diff --git a/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java b/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java index 17f12b331259..02b3d241d893 100644 --- a/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java +++ b/api/src/main/java/org/apache/iceberg/BatchScanAdapter.java @@ -83,6 +83,11 @@ public BatchScan includeColumnStats() { return new BatchScanAdapter(scan.includeColumnStats()); } + @Override + public BatchScan includeColumnStats(Collection requestedColumns) { + return new BatchScanAdapter(scan.includeColumnStats(requestedColumns)); + } + @Override public BatchScan select(Collection columns) { return new BatchScanAdapter(scan.select(columns)); diff --git a/api/src/main/java/org/apache/iceberg/ContentFile.java b/api/src/main/java/org/apache/iceberg/ContentFile.java index fd8f7de62f1e..4bb8a78289de 100644 --- a/api/src/main/java/org/apache/iceberg/ContentFile.java +++ b/api/src/main/java/org/apache/iceberg/ContentFile.java @@ -21,6 +21,7 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.Set; /** * Superinterface of {@link DataFile} and {@link DeleteFile} that exposes common methods. @@ -165,6 +166,20 @@ default Long fileSequenceNumber() { */ F copyWithoutStats(); + /** + * Copies this file with column stats only for specific columns. Manifest readers can reuse file + * instances; use this method to copy data with stats only for specific columns when collecting + * files. + * + * @param requestedColumnIds column IDs for which to keep stats. + * @return a copy of data file, with lower bounds, upper bounds, value counts, null value counts, + * and nan value counts for only specific columns. + */ + default F copyWithStats(Set requestedColumnIds) { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement copyWithStats"); + } + /** * Copies this file (potentially without file stats). Manifest readers can reuse file instances; * use this method to copy data when collecting files from tasks. diff --git a/api/src/main/java/org/apache/iceberg/Scan.java b/api/src/main/java/org/apache/iceberg/Scan.java index deebb66f2a24..339bc75336ba 100644 --- a/api/src/main/java/org/apache/iceberg/Scan.java +++ b/api/src/main/java/org/apache/iceberg/Scan.java @@ -77,6 +77,20 @@ public interface Scan> { */ ThisT includeColumnStats(); + /** + * Create a new scan from this that loads the column stats for the specific columns with each data + * file. + * + *

Column stats include: value count, null value count, lower bounds, and upper bounds. + * + * @param requestedColumns column names for which to keep the stats. + * @return a new scan based on this that loads column stats for specific columns. + */ + default ThisT includeColumnStats(Collection requestedColumns) { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement includeColumnStats"); + } + /** * Create a new scan from this that will read the given data columns. This produces an expected * schema that includes all fields that are either selected or used by this scan's filter diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index 890ae8abd474..cefd765a91b5 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -38,6 +38,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.IntStream; import org.apache.iceberg.expressions.BoundPredicate; import org.apache.iceberg.expressions.BoundSetPredicate; @@ -666,6 +667,11 @@ public DataFile copyWithoutStats() { return this; } + @Override + public DataFile copyWithStats(Set requestedColumns) { + return this; + } + @Override public List splitOffsets() { return null; diff --git a/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java b/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java index eca5e7d95c13..263830a53b3c 100644 --- a/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java @@ -38,6 +38,7 @@ import org.apache.iceberg.metrics.ScanMetricsUtil; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.ParallelIterable; import org.apache.iceberg.util.TableScanUtil; import org.apache.iceberg.util.ThreadPools; @@ -368,7 +369,7 @@ private CloseableIterable toFileTasks( ScanMetricsUtil.fileTask(scanMetrics(), dataFile, deleteFiles); return new BaseFileScanTask( - copyDataFiles ? dataFile.copy(shouldReturnColumnStats()) : dataFile, + copyDataFiles ? copy(dataFile) : dataFile, deleteFiles, schemaString, specString, @@ -376,6 +377,10 @@ private CloseableIterable toFileTasks( }); } + private > F copy(F file) { + return ContentFileUtil.copy(file, shouldReturnColumnStats(), columnsToKeepStats()); + } + private ManifestEvaluator newManifestEvaluator(PartitionSpec spec) { Expression projection = Projections.inclusive(spec, isCaseSensitive()).project(filter()); return ManifestEvaluator.forPartitionFilter(projection, spec, isCaseSensitive()); diff --git a/core/src/main/java/org/apache/iceberg/BaseFile.java b/core/src/main/java/org/apache/iceberg/BaseFile.java index daa209d0b401..6fe2fcc149cb 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFile.java +++ b/core/src/main/java/org/apache/iceberg/BaseFile.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Set; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.avro.specific.SpecificData; @@ -170,9 +171,11 @@ public PartitionData copy() { * Copy constructor. * * @param toCopy a generic data file to copy. - * @param fullCopy whether to copy all fields or to drop column-level stats + * @param copyStats whether to copy all fields or to drop column-level stats + * @param requestedColumnIds column ids for which to keep stats. If null then every + * column stat is kept. */ - BaseFile(BaseFile toCopy, boolean fullCopy) { + BaseFile(BaseFile toCopy, boolean copyStats, Set requestedColumnIds) { this.fileOrdinal = toCopy.fileOrdinal; this.partitionSpecId = toCopy.partitionSpecId; this.content = toCopy.content; @@ -182,13 +185,13 @@ public PartitionData copy() { this.partitionType = toCopy.partitionType; this.recordCount = toCopy.recordCount; this.fileSizeInBytes = toCopy.fileSizeInBytes; - if (fullCopy) { - this.columnSizes = SerializableMap.copyOf(toCopy.columnSizes); - this.valueCounts = SerializableMap.copyOf(toCopy.valueCounts); - this.nullValueCounts = SerializableMap.copyOf(toCopy.nullValueCounts); - this.nanValueCounts = SerializableMap.copyOf(toCopy.nanValueCounts); - this.lowerBounds = SerializableByteBufferMap.wrap(SerializableMap.copyOf(toCopy.lowerBounds)); - this.upperBounds = SerializableByteBufferMap.wrap(SerializableMap.copyOf(toCopy.upperBounds)); + if (copyStats) { + this.columnSizes = copyMap(toCopy.columnSizes, requestedColumnIds); + this.valueCounts = copyMap(toCopy.valueCounts, requestedColumnIds); + this.nullValueCounts = copyMap(toCopy.nullValueCounts, requestedColumnIds); + this.nanValueCounts = copyMap(toCopy.nanValueCounts, requestedColumnIds); + this.lowerBounds = copyByteBufferMap(toCopy.lowerBounds, requestedColumnIds); + this.upperBounds = copyByteBufferMap(toCopy.upperBounds, requestedColumnIds); } else { this.columnSizes = null; this.valueCounts = null; @@ -493,6 +496,15 @@ public Integer sortOrderId() { return sortOrderId; } + private static Map copyMap(Map map, Set keys) { + return keys == null ? SerializableMap.copyOf(map) : SerializableMap.filteredCopyOf(map, keys); + } + + private static Map copyByteBufferMap( + Map map, Set keys) { + return SerializableByteBufferMap.wrap(copyMap(map, keys)); + } + private static Map toReadableMap(Map map) { if (map == null) { return null; diff --git a/core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java b/core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java index 6d56ce5d932a..24b9ae1acead 100644 --- a/core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java @@ -83,7 +83,8 @@ private CloseableIterable appendFilesFromSnapshots(List snapshotIds.contains(manifestEntry.snapshotId()) && manifestEntry.status() == ManifestEntry.Status.ADDED) .specsById(table().specs()) - .ignoreDeleted(); + .ignoreDeleted() + .columnsToKeepStats(columnsToKeepStats()); if (context().ignoreResiduals()) { manifestGroup = manifestGroup.ignoreResiduals(); diff --git a/core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java b/core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java index 308e3b4646b0..2d54a94e8d73 100644 --- a/core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java @@ -78,7 +78,8 @@ protected CloseableIterable doPlanFiles( .select(scanColumns()) .filterData(filter()) .filterManifestEntries(entry -> changelogSnapshotIds.contains(entry.snapshotId())) - .ignoreExisting(); + .ignoreExisting() + .columnsToKeepStats(columnsToKeepStats()); if (shouldIgnoreResiduals()) { manifestGroup = manifestGroup.ignoreResiduals(); diff --git a/core/src/main/java/org/apache/iceberg/BaseScan.java b/core/src/main/java/org/apache/iceberg/BaseScan.java index 322c48bc9ab0..8c309cf69e6d 100644 --- a/core/src/main/java/org/apache/iceberg/BaseScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseScan.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; import org.apache.iceberg.expressions.Binder; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -122,6 +123,10 @@ protected boolean shouldReturnColumnStats() { return context().returnColumnStats(); } + protected Set columnsToKeepStats() { + return context().columnsToKeepStats(); + } + protected boolean shouldIgnoreResiduals() { return context().ignoreResiduals(); } @@ -166,6 +171,19 @@ public ThisT includeColumnStats() { return newRefinedScan(table, schema, context.shouldReturnColumnStats(true)); } + @Override + public ThisT includeColumnStats(Collection requestedColumns) { + return newRefinedScan( + table, + schema, + context + .shouldReturnColumnStats(true) + .columnsToKeepStats( + requestedColumns.stream() + .map(c -> schema.findField(c).fieldId()) + .collect(Collectors.toSet()))); + } + @Override public ThisT select(Collection columns) { return newRefinedScan(table, schema, context.selectColumns(columns)); diff --git a/core/src/main/java/org/apache/iceberg/DataScan.java b/core/src/main/java/org/apache/iceberg/DataScan.java index 8de48740b9bf..1c48042f52f0 100644 --- a/core/src/main/java/org/apache/iceberg/DataScan.java +++ b/core/src/main/java/org/apache/iceberg/DataScan.java @@ -55,7 +55,8 @@ protected ManifestGroup newManifestGroup( .filterData(filter()) .specsById(table().specs()) .scanMetrics(scanMetrics()) - .ignoreDeleted(); + .ignoreDeleted() + .columnsToKeepStats(columnsToKeepStats()); if (shouldIgnoreResiduals()) { manifestGroup = manifestGroup.ignoreResiduals(); diff --git a/core/src/main/java/org/apache/iceberg/DataTableScan.java b/core/src/main/java/org/apache/iceberg/DataTableScan.java index 67d2b0ef35e3..8463112b7a51 100644 --- a/core/src/main/java/org/apache/iceberg/DataTableScan.java +++ b/core/src/main/java/org/apache/iceberg/DataTableScan.java @@ -76,7 +76,8 @@ public CloseableIterable doPlanFiles() { .filterData(filter()) .specsById(table().specs()) .scanMetrics(scanMetrics()) - .ignoreDeleted(); + .ignoreDeleted() + .columnsToKeepStats(columnsToKeepStats()); if (shouldIgnoreResiduals()) { manifestGroup = manifestGroup.ignoreResiduals(); diff --git a/core/src/main/java/org/apache/iceberg/GenericDataFile.java b/core/src/main/java/org/apache/iceberg/GenericDataFile.java index 07c5172f1b3f..f78bb2ce9f2e 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDataFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import java.util.List; +import java.util.Set; import org.apache.avro.Schema; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -66,10 +67,13 @@ class GenericDataFile extends BaseFile implements DataFile { * Copy constructor. * * @param toCopy a generic data file to copy. - * @param fullCopy whether to copy all fields or to drop column-level stats + * @param copyStats whether to copy all fields or to drop column-level stats. + * @param requestedColumnIds column ids for which to keep stats. If null then every + * column stat is kept. */ - private GenericDataFile(GenericDataFile toCopy, boolean fullCopy) { - super(toCopy, fullCopy); + private GenericDataFile( + GenericDataFile toCopy, boolean copyStats, Set requestedColumnIds) { + super(toCopy, copyStats, requestedColumnIds); } /** Constructor for Java serialization. */ @@ -77,12 +81,17 @@ private GenericDataFile(GenericDataFile toCopy, boolean fullCopy) { @Override public DataFile copyWithoutStats() { - return new GenericDataFile(this, false /* drop stats */); + return new GenericDataFile(this, false /* drop stats */, null); + } + + @Override + public DataFile copyWithStats(Set requestedColumnIds) { + return new GenericDataFile(this, true, requestedColumnIds); } @Override public DataFile copy() { - return new GenericDataFile(this, true /* full copy */); + return new GenericDataFile(this, true /* full copy */, null); } @Override diff --git a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java index cf20d931aa28..509bf4c16b03 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import java.util.List; +import java.util.Set; import org.apache.avro.Schema; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -67,10 +68,13 @@ class GenericDeleteFile extends BaseFile implements DeleteFile { * Copy constructor. * * @param toCopy a generic data file to copy. - * @param fullCopy whether to copy all fields or to drop column-level stats + * @param copyStats whether to copy all fields or to drop column-level stats. + * @param requestedColumnIds column ids for which to keep stats. If null then every + * column stat is kept. */ - private GenericDeleteFile(GenericDeleteFile toCopy, boolean fullCopy) { - super(toCopy, fullCopy); + private GenericDeleteFile( + GenericDeleteFile toCopy, boolean copyStats, Set requestedColumnIds) { + super(toCopy, copyStats, requestedColumnIds); } /** Constructor for Java serialization. */ @@ -78,12 +82,17 @@ private GenericDeleteFile(GenericDeleteFile toCopy, boolean fullCopy) { @Override public DeleteFile copyWithoutStats() { - return new GenericDeleteFile(this, false /* drop stats */); + return new GenericDeleteFile(this, false /* drop stats */, null); + } + + @Override + public DeleteFile copyWithStats(Set requestedColumnIds) { + return new GenericDeleteFile(this, true, requestedColumnIds); } @Override public DeleteFile copy() { - return new GenericDeleteFile(this, true /* full copy */); + return new GenericDeleteFile(this, true /* full copy */, null); } @Override diff --git a/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java b/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java index 672c268aad56..408e049ea65c 100644 --- a/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java +++ b/core/src/main/java/org/apache/iceberg/IncrementalDataTableScan.java @@ -102,7 +102,8 @@ public CloseableIterable planFiles() { snapshotIds.contains(manifestEntry.snapshotId()) && manifestEntry.status() == ManifestEntry.Status.ADDED) .specsById(table().specs()) - .ignoreDeleted(); + .ignoreDeleted() + .columnsToKeepStats(columnsToKeepStats()); if (shouldIgnoreResiduals()) { manifestGroup = manifestGroup.ignoreResiduals(); diff --git a/core/src/main/java/org/apache/iceberg/ManifestGroup.java b/core/src/main/java/org/apache/iceberg/ManifestGroup.java index 027b8764a9d8..97e480837c7b 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestGroup.java +++ b/core/src/main/java/org/apache/iceberg/ManifestGroup.java @@ -42,6 +42,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.ParallelIterable; class ManifestGroup { @@ -61,6 +62,7 @@ class ManifestGroup { private boolean ignoreResiduals; private List columns; private boolean caseSensitive; + private Set columnsToKeepStats; private ExecutorService executorService; private ScanMetrics scanMetrics; @@ -154,6 +156,12 @@ ManifestGroup caseSensitive(boolean newCaseSensitive) { return this; } + ManifestGroup columnsToKeepStats(Set newColumnsToKeepStats) { + this.columnsToKeepStats = + newColumnsToKeepStats == null ? null : Sets.newHashSet(newColumnsToKeepStats); + return this; + } + ManifestGroup planWith(ExecutorService newExecutorService) { this.executorService = newExecutorService; deleteIndexBuilder.planWith(newExecutorService); @@ -193,7 +201,8 @@ public CloseableIterable plan(CreateTasksFunction cre specId -> { PartitionSpec spec = specsById.get(specId); ResidualEvaluator residuals = residualCache.get(specId); - return new TaskContext(spec, deleteFiles, residuals, dropStats, scanMetrics); + return new TaskContext( + spec, deleteFiles, residuals, dropStats, columnsToKeepStats, scanMetrics); }); Iterable> tasks = @@ -361,7 +370,8 @@ private static CloseableIterable createFileScanTasks( return CloseableIterable.transform( entries, entry -> { - DataFile dataFile = entry.file().copy(ctx.shouldKeepStats()); + DataFile dataFile = + ContentFileUtil.copy(entry.file(), ctx.shouldKeepStats(), ctx.columnsToKeepStats()); DeleteFile[] deleteFiles = ctx.deletes().forEntry(entry); ScanMetricsUtil.fileTask(ctx.scanMetrics(), dataFile, deleteFiles); return new BaseFileScanTask( @@ -381,6 +391,7 @@ static class TaskContext { private final DeleteFileIndex deletes; private final ResidualEvaluator residuals; private final boolean dropStats; + private final Set columnsToKeepStats; private final ScanMetrics scanMetrics; TaskContext( @@ -388,12 +399,14 @@ static class TaskContext { DeleteFileIndex deletes, ResidualEvaluator residuals, boolean dropStats, + Set columnsToKeepStats, ScanMetrics scanMetrics) { this.schemaAsString = SchemaParser.toJson(spec.schema()); this.specAsString = PartitionSpecParser.toJson(spec); this.deletes = deletes; this.residuals = residuals; this.dropStats = dropStats; + this.columnsToKeepStats = columnsToKeepStats; this.scanMetrics = scanMetrics; } @@ -417,6 +430,10 @@ boolean shouldKeepStats() { return !dropStats; } + Set columnsToKeepStats() { + return columnsToKeepStats; + } + public ScanMetrics scanMetrics() { return scanMetrics; } diff --git a/core/src/main/java/org/apache/iceberg/TableScanContext.java b/core/src/main/java/org/apache/iceberg/TableScanContext.java index b59446a717b7..5722ed7d8c1c 100644 --- a/core/src/main/java/org/apache/iceberg/TableScanContext.java +++ b/core/src/main/java/org/apache/iceberg/TableScanContext.java @@ -20,6 +20,7 @@ import java.util.Collection; import java.util.Map; +import java.util.Set; import java.util.concurrent.ExecutorService; import javax.annotation.Nullable; import org.apache.iceberg.expressions.Expression; @@ -59,6 +60,9 @@ public boolean returnColumnStats() { return false; } + @Nullable + public abstract Set columnsToKeepStats(); + @Nullable public abstract Collection selectedColumns(); @@ -125,6 +129,16 @@ TableScanContext shouldReturnColumnStats(boolean returnColumnStats) { .build(); } + TableScanContext columnsToKeepStats(Set columnsToKeepStats) { + Preconditions.checkState( + returnColumnStats(), + "Cannot select columns to keep stats when column stats are not returned"); + return ImmutableTableScanContext.builder() + .from(this) + .columnsToKeepStats(columnsToKeepStats) + .build(); + } + TableScanContext selectColumns(Collection columns) { Preconditions.checkState( projectedSchema() == null, "Cannot select columns when projection schema is set"); diff --git a/core/src/main/java/org/apache/iceberg/V1Metadata.java b/core/src/main/java/org/apache/iceberg/V1Metadata.java index f262614127bd..fddcee037488 100644 --- a/core/src/main/java/org/apache/iceberg/V1Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V1Metadata.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.avro.generic.IndexedRecord; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.types.Types; @@ -485,6 +486,11 @@ public DataFile copy() { return wrapped.copy(); } + @Override + public DataFile copyWithStats(Set requestedColumnIds) { + return wrapped.copyWithStats(requestedColumnIds); + } + @Override public DataFile copyWithoutStats() { return wrapped.copyWithoutStats(); diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index 7db54e46b95e..ba6bb4a31960 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Map; +import java.util.Set; import org.apache.avro.generic.IndexedRecord; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -560,6 +561,11 @@ public F copy() { throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); } + @Override + public F copyWithStats(Set requestedColumnIds) { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + @Override public F copyWithoutStats() { throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); diff --git a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java new file mode 100644 index 000000000000..d027567e069c --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import java.util.Set; +import org.apache.iceberg.ContentFile; + +public class ContentFileUtil { + private ContentFileUtil() {} + + /** + * Copies the {@link ContentFile} with the specific stat settings. + * + * @param file a generic data file to copy. + * @param withStats whether to keep any stats + * @param requestedColumnIds column ids for which to keep stats. If null then every + * column stat is kept. + * @return The copied file + */ + public static , K> K copy( + F file, boolean withStats, Set requestedColumnIds) { + if (withStats) { + return requestedColumnIds != null ? file.copyWithStats(requestedColumnIds) : file.copy(); + } else { + return file.copyWithoutStats(); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/SerializableMap.java b/core/src/main/java/org/apache/iceberg/util/SerializableMap.java index 137f915c3a2c..908829daeae2 100644 --- a/core/src/main/java/org/apache/iceberg/util/SerializableMap.java +++ b/core/src/main/java/org/apache/iceberg/util/SerializableMap.java @@ -26,6 +26,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; public class SerializableMap implements Map, Serializable { + private static final long serialVersionUID = -3377238354349859240L; private final Map copiedMap; private transient volatile Map immutableMap; @@ -39,10 +40,26 @@ private SerializableMap(Map map) { this.copiedMap.putAll(map); } + private SerializableMap(Map map, Set keys) { + Map filteredMap = Maps.newHashMapWithExpectedSize(keys.size()); + + for (K key : keys) { + if (map.containsKey(key)) { + filteredMap.put(key, map.get(key)); + } + } + + this.copiedMap = filteredMap; + } + public static SerializableMap copyOf(Map map) { return map == null ? null : new SerializableMap<>(map); } + public static SerializableMap filteredCopyOf(Map map, Set keys) { + return map == null ? null : new SerializableMap<>(map, keys); + } + public Map immutableMap() { if (immutableMap == null) { synchronized (this) { diff --git a/core/src/test/java/org/apache/iceberg/TestScanDataFileColumns.java b/core/src/test/java/org/apache/iceberg/TestScanDataFileColumns.java index cfe5796c2695..8baec3184fd7 100644 --- a/core/src/test/java/org/apache/iceberg/TestScanDataFileColumns.java +++ b/core/src/test/java/org/apache/iceberg/TestScanDataFileColumns.java @@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.types.Types; import org.junit.Assert; import org.junit.Before; @@ -70,12 +71,12 @@ public void createTables() throws IOException { .withMetrics( new Metrics( 3L, - ImmutableMap.of(1, 50L), // column size - ImmutableMap.of(1, 3L), // value count - ImmutableMap.of(1, 0L), // null count + ImmutableMap.of(1, 50L, 2, 100L), // column size + ImmutableMap.of(1, 3L, 2, 5L), // value count + ImmutableMap.of(1, 0L, 2, 0L), // null count null, - ImmutableMap.of(1, longToBuffer(0L)), // lower bounds - ImmutableMap.of(1, longToBuffer(2L)))) // upper bounds) + ImmutableMap.of(1, longToBuffer(0L), 2, longToBuffer(3L)), // lower bounds + ImmutableMap.of(1, longToBuffer(2L), 2, longToBuffer(4L)))) // upper bounds .build()) .appendFile( DataFiles.builder(PartitionSpec.unpartitioned()) @@ -84,12 +85,13 @@ public void createTables() throws IOException { .withMetrics( new Metrics( 3L, - ImmutableMap.of(1, 60L), // column size - ImmutableMap.of(1, 3L), // value count - ImmutableMap.of(1, 0L), // null count + ImmutableMap.of(1, 60L, 2, 120L), // column size + ImmutableMap.of(1, 3L, 2, 10L), // value count + ImmutableMap.of(1, 0L, 2, 0L), // null count null, - ImmutableMap.of(1, longToBuffer(10L)), // lower bounds - ImmutableMap.of(1, longToBuffer(12L)))) // upper bounds) + ImmutableMap.of(1, longToBuffer(10L), 2, longToBuffer(13L)), // lower bounds + ImmutableMap.of( + 1, longToBuffer(12L), 2, longToBuffer(14L)))) // upper bounds .build()) .appendFile( DataFiles.builder(PartitionSpec.unpartitioned()) @@ -98,12 +100,13 @@ public void createTables() throws IOException { .withMetrics( new Metrics( 3L, - ImmutableMap.of(1, 70L), // column size - ImmutableMap.of(1, 3L), // value count - ImmutableMap.of(1, 0L), // null count + ImmutableMap.of(1, 70L, 2, 140L), // column size + ImmutableMap.of(1, 3L, 2, 15L), // value count + ImmutableMap.of(1, 0L, 2, 0L), // null count null, - ImmutableMap.of(1, longToBuffer(20L)), // lower bounds - ImmutableMap.of(1, longToBuffer(22L)))) // upper bounds) + ImmutableMap.of(1, longToBuffer(20L), 2, longToBuffer(23L)), // lower bounds + ImmutableMap.of( + 1, longToBuffer(22L), 2, longToBuffer(24L)))) // upper bounds .build()) .commit(); } @@ -123,11 +126,24 @@ public void testColumnStatsIgnored() { public void testColumnStatsLoading() { // stats columns should be suppressed by default for (FileScanTask fileTask : table.newScan().includeColumnStats().planFiles()) { - Assert.assertNotNull(fileTask.file().valueCounts()); - Assert.assertNotNull(fileTask.file().nullValueCounts()); - Assert.assertNotNull(fileTask.file().lowerBounds()); - Assert.assertNotNull(fileTask.file().upperBounds()); - Assert.assertNotNull(fileTask.file().columnSizes()); + Assert.assertEquals(2, fileTask.file().valueCounts().size()); + Assert.assertEquals(2, fileTask.file().nullValueCounts().size()); + Assert.assertEquals(2, fileTask.file().lowerBounds().size()); + Assert.assertEquals(2, fileTask.file().upperBounds().size()); + Assert.assertEquals(2, fileTask.file().columnSizes().size()); + } + } + + @Test + public void testColumnStatsPartial() { + // stats columns should be suppressed by default + for (FileScanTask fileTask : + table.newScan().includeColumnStats(ImmutableSet.of("id")).planFiles()) { + Assert.assertEquals(1, fileTask.file().valueCounts().size()); + Assert.assertEquals(1, fileTask.file().nullValueCounts().size()); + Assert.assertEquals(1, fileTask.file().lowerBounds().size()); + Assert.assertEquals(1, fileTask.file().upperBounds().size()); + Assert.assertEquals(1, fileTask.file().columnSizes().size()); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java index ea317e93d8ba..15078809714f 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java @@ -165,6 +165,10 @@ private static > T refineScanW refinedScan = refinedScan.includeColumnStats(); } + if (context.includeStatsForColumns() != null) { + refinedScan = refinedScan.includeColumnStats(context.includeStatsForColumns()); + } + refinedScan = refinedScan.option(TableProperties.SPLIT_SIZE, context.splitSize().toString()); refinedScan = diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java index e380204e871f..4357b1f57df6 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.time.Duration; +import java.util.Collection; import java.util.List; import java.util.Map; import org.apache.flink.annotation.Internal; @@ -62,6 +63,7 @@ public class ScanContext implements Serializable { private final List filters; private final long limit; private final boolean includeColumnStats; + private final Collection includeStatsForColumns; private final Integer planParallelism; private final int maxPlanningSnapshotCount; private final int maxAllowedPlanningFailures; @@ -84,6 +86,7 @@ private ScanContext( List filters, long limit, boolean includeColumnStats, + Collection includeStatsForColumns, boolean exposeLocality, Integer planParallelism, int maxPlanningSnapshotCount, @@ -114,6 +117,7 @@ private ScanContext( this.filters = filters; this.limit = limit; this.includeColumnStats = includeColumnStats; + this.includeStatsForColumns = includeStatsForColumns; this.exposeLocality = exposeLocality; this.planParallelism = planParallelism; this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; @@ -248,6 +252,10 @@ public boolean includeColumnStats() { return includeColumnStats; } + public Collection includeStatsForColumns() { + return includeStatsForColumns; + } + public boolean exposeLocality() { return exposeLocality; } @@ -285,6 +293,7 @@ public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSn .filters(filters) .limit(limit) .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) .exposeLocality(exposeLocality) .planParallelism(planParallelism) .maxPlanningSnapshotCount(maxPlanningSnapshotCount) @@ -313,6 +322,7 @@ public ScanContext copyWithSnapshotId(long newSnapshotId) { .filters(filters) .limit(limit) .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) .exposeLocality(exposeLocality) .planParallelism(planParallelism) .maxPlanningSnapshotCount(maxPlanningSnapshotCount) @@ -349,6 +359,7 @@ public static class Builder { private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); private boolean includeColumnStats = FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); + private Collection includeStatsForColumns = null; private boolean exposeLocality; private Integer planParallelism = FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); @@ -464,6 +475,11 @@ public Builder includeColumnStats(boolean newIncludeColumnStats) { return this; } + public Builder includeColumnStats(Collection newIncludeStatsForColumns) { + this.includeStatsForColumns = newIncludeStatsForColumns; + return this; + } + public Builder exposeLocality(boolean newExposeLocality) { this.exposeLocality = newExposeLocality; return this; @@ -531,6 +547,7 @@ public ScanContext build() { filters, limit, includeColumnStats, + includeStatsForColumns, exposeLocality, planParallelism, maxPlanningSnapshotCount, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 2fa921b1f112..1bb2b1c6bf85 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -84,7 +84,7 @@ private void appendTwoSnapshots() throws IOException { } /** @return the last enumerated snapshot id */ - private IcebergEnumeratorPosition verifyOneCycle( + private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { List batch = @@ -106,7 +106,7 @@ private IcebergEnumeratorPosition verifyOneCycle( Assert.assertEquals( dataFile.path().toString(), Iterables.getOnlyElement(split.task().files()).file().path().toString()); - return result.toPosition(); + return new CycleResult(result.toPosition(), split); } @Test @@ -135,7 +135,7 @@ public void testTableScanThenIncrementalWithEmptyTable() throws Exception { // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -169,7 +169,7 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -206,7 +206,7 @@ public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { // next 3 snapshots IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -251,7 +251,7 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -281,7 +281,7 @@ public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -323,12 +323,12 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @Test - public void testIncrementalFromSnapshotIdWithEmptyTable() throws Exception { + public void testIncrementalFromSnapshotIdWithEmptyTable() { ScanContext scanContextWithInvalidSnapshotId = ScanContext.builder() .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) @@ -411,12 +411,12 @@ public void testIncrementalFromSnapshotId() throws Exception { IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @Test - public void testIncrementalFromSnapshotTimestampWithEmptyTable() throws Exception { + public void testIncrementalFromSnapshotTimestampWithEmptyTable() { ScanContext scanContextWithInvalidSnapshotId = ScanContext.builder() .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) @@ -493,7 +493,7 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -533,6 +533,115 @@ public void testMaxPlanningSnapshotCount() throws Exception { thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); } + @Test + public void testTableScanNoStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(false) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 0); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 0); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanAllStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 3); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 3); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanSingleStat() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(ImmutableSet.of("data")) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 1); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 1); + lastPosition = result.lastPosition; + } + } + + private void verifyStatCount(IcebergSourceSplit split, int expected) { + if (expected == 0) { + split + .task() + .files() + .forEach( + f -> { + Assert.assertNull(f.file().valueCounts()); + Assert.assertNull(f.file().columnSizes()); + Assert.assertNull(f.file().lowerBounds()); + Assert.assertNull(f.file().upperBounds()); + Assert.assertNull(f.file().nanValueCounts()); + Assert.assertNull(f.file().nullValueCounts()); + }); + } else { + split + .task() + .files() + .forEach( + f -> { + Assert.assertEquals(expected, f.file().valueCounts().size()); + Assert.assertEquals(expected, f.file().columnSizes().size()); + Assert.assertEquals(expected, f.file().lowerBounds().size()); + Assert.assertEquals(expected, f.file().upperBounds().size()); + Assert.assertEquals(expected, f.file().nullValueCounts().size()); + // The nanValue is not counted for long and string fields + Assert.assertEquals(0, f.file().nanValueCounts().size()); + }); + } + } + private void verifyMaxPlanningSnapshotCountResult( ContinuousEnumerationResult result, Snapshot fromSnapshotExclusive, @@ -570,4 +679,14 @@ private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { dataAppender.appendToTable(dataFile); return tableResource.table().currentSnapshot(); } + + private static class CycleResult { + IcebergEnumeratorPosition lastPosition; + IcebergSourceSplit split; + + CycleResult(IcebergEnumeratorPosition lastPosition, IcebergSourceSplit split) { + this.lastPosition = lastPosition; + this.split = split; + } + } } From bfe1d03c75b4a8bd2c15c925f39cdec28c312489 Mon Sep 17 00:00:00 2001 From: zhen Date: Thu, 16 Nov 2023 12:52:47 +0800 Subject: [PATCH 34/94] Spark 3.5: Support metadata columns in staged scan (#8872) --- ...TestMetaColumnProjectionWithStageScan.java | 127 ++++++++++++++++++ .../iceberg/spark/source/SparkStagedScan.java | 10 +- .../spark/source/SparkStagedScanBuilder.java | 52 ++++++- 3 files changed, 183 insertions(+), 6 deletions(-) create mode 100644 spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java new file mode 100644 index 000000000000..e9013848cf11 --- /dev/null +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Test; +import org.junit.runners.Parameterized; + +public class TestMetaColumnProjectionWithStageScan extends SparkExtensionsTestBase { + + public TestMetaColumnProjectionWithStageScan( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties() + } + }; + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + private void stageTask( + Table tab, String fileSetID, CloseableIterable tasks) { + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + taskSetManager.stageTasks(tab, fileSetID, Lists.newArrayList(tasks)); + } + + @Test + public void testReadStageTableMeta() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" + + "('format-version'='2', 'write.delete.mode'='merge-on-read')", + tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + table.refresh(); + String tableLocation = table.location(); + + try (CloseableIterable tasks = table.newBatchScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(table, fileSetID, tasks); + Dataset scanDF2 = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .load(tableLocation); + + Assertions.assertThat(scanDF2.columns().length).isEqualTo(2); + } + + try (CloseableIterable tasks = table.newBatchScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(table, fileSetID, tasks); + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .load(tableLocation) + .select("*", "_pos"); + + List rows = scanDF.collectAsList(); + ImmutableList expectedRows = + ImmutableList.of(row(1L, "a", 0L), row(2L, "b", 1L), row(3L, "c", 2L), row(4L, "d", 3L)); + assertEquals("result should match", expectedRows, rowsToJava(rows)); + } + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java index 0290bf7e84ce..fd299ade7fdc 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java @@ -22,6 +22,7 @@ import java.util.Objects; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -39,9 +40,8 @@ class SparkStagedScan extends SparkScan { private List> taskGroups = null; // lazy cache of tasks - SparkStagedScan(SparkSession spark, Table table, SparkReadConf readConf) { - super(spark, table, readConf, table.schema(), ImmutableList.of(), null); - + SparkStagedScan(SparkSession spark, Table table, Schema expectedSchema, SparkReadConf readConf) { + super(spark, table, readConf, expectedSchema, ImmutableList.of(), null); this.taskSetId = readConf.scanTaskSetId(); this.splitSize = readConf.splitSize(); this.splitLookback = readConf.splitLookback(); @@ -77,6 +77,7 @@ public boolean equals(Object other) { SparkStagedScan that = (SparkStagedScan) other; return table().name().equals(that.table().name()) && Objects.equals(taskSetId, that.taskSetId) + && readSchema().equals(that.readSchema()) && Objects.equals(splitSize, that.splitSize) && Objects.equals(splitLookback, that.splitLookback) && Objects.equals(openFileCost, that.openFileCost); @@ -84,7 +85,8 @@ public boolean equals(Object other) { @Override public int hashCode() { - return Objects.hash(table().name(), taskSetId, splitSize, splitSize, openFileCost); + return Objects.hash( + table().name(), taskSetId, readSchema(), splitSize, splitSize, openFileCost); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index 37bbea42e5b1..25393888f95c 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -18,27 +18,75 @@ */ package org.apache.iceberg.spark.source; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -class SparkStagedScanBuilder implements ScanBuilder { +class SparkStagedScanBuilder implements ScanBuilder, SupportsPushDownRequiredColumns { private final SparkSession spark; private final Table table; private final SparkReadConf readConf; + private final List metaColumns = Lists.newArrayList(); + + private Schema schema = null; SparkStagedScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { this.spark = spark; this.table = table; this.readConf = new SparkReadConf(spark, table, options); + this.schema = table.schema(); } @Override public Scan build() { - return new SparkStagedScan(spark, table, readConf); + return new SparkStagedScan(spark, table, schemaWithMetadataColumns(), readConf); + } + + @Override + public void pruneColumns(StructType requestedSchema) { + StructType requestedProjection = removeMetaColumns(requestedSchema); + this.schema = SparkSchemaUtil.prune(schema, requestedProjection); + + Stream.of(requestedSchema.fields()) + .map(StructField::name) + .filter(MetadataColumns::isMetadataColumn) + .distinct() + .forEach(metaColumns::add); + } + + private StructType removeMetaColumns(StructType structType) { + return new StructType( + Stream.of(structType.fields()) + .filter(field -> MetadataColumns.nonMetadataColumn(field.name())) + .toArray(StructField[]::new)); + } + + private Schema schemaWithMetadataColumns() { + // metadata columns + List fields = + metaColumns.stream() + .distinct() + .map(name -> MetadataColumns.metadataColumn(table, name)) + .collect(Collectors.toList()); + Schema meta = new Schema(fields); + + // schema of rows returned by readers + return TypeUtil.join(schema, meta); } } From 72da856b3c9ba2da92a5aa5d8e69afdf9a3c24b0 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Fri, 17 Nov 2023 00:38:24 +0900 Subject: [PATCH 35/94] Docs: Fix parquet default compression codec (#9096) --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index e364b9c939f4..ead5c763ab2b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -55,7 +55,7 @@ Iceberg tables support table properties to configure table behavior, like the de | write.parquet.page-size-bytes | 1048576 (1 MB) | Parquet page size | | write.parquet.page-row-limit | 20000 | Parquet page row limit | | write.parquet.dict-size-bytes | 2097152 (2 MB) | Parquet dictionary page size | -| write.parquet.compression-codec | gzip | Parquet compression codec: zstd, brotli, lz4, gzip, snappy, uncompressed | +| write.parquet.compression-codec | zstd | Parquet compression codec: zstd, brotli, lz4, gzip, snappy, uncompressed | | write.parquet.compression-level | null | Parquet compression level | | write.parquet.bloom-filter-enabled.column.col1 | (not set) | Hint to parquet to write a bloom filter for the column: col1 | | write.parquet.bloom-filter-max-bytes | 1048576 (1 MB) | The maximum number of bytes for a bloom filter bitset | From 798f1c8abe3ed24a2229d4f3d1b5d7c009a7fa7e Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Thu, 16 Nov 2023 16:39:54 +0100 Subject: [PATCH 36/94] Azure: Allow shared-key auth for testing purposes (#9068) The Azure Blob Storage client library, including the hadoop-azure stuff, allow using shared-key authentication, which is handy for testing purposes with Azurite. Unfortunately, `AzureProperties` does not play nicely here, because it does not support shared-keys. This change adds two new properties to allow shared-key authentication. --- .../apache/iceberg/azure/AzureProperties.java | 19 +++++++++ .../iceberg/azure/AzurePropertiesTest.java | 41 +++++++++++++++++++ 2 files changed, 60 insertions(+) diff --git a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java index 01e3d2e12143..8fcb0a098fbf 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -19,11 +19,13 @@ package org.apache.iceberg.azure; import com.azure.identity.DefaultAzureCredentialBuilder; +import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import java.io.Serializable; import java.util.Collections; import java.util.Map; import java.util.Optional; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.PropertyUtil; public class AzureProperties implements Serializable { @@ -31,9 +33,12 @@ public class AzureProperties implements Serializable { public static final String ADLS_CONNECTION_STRING_PREFIX = "adls.connection-string."; public static final String ADLS_READ_BLOCK_SIZE = "adls.read.block-size-bytes"; public static final String ADLS_WRITE_BLOCK_SIZE = "adls.write.block-size-bytes"; + public static final String ADLS_SHARED_KEY_ACCOUNT_NAME = "adls.auth.shared-key.account.name"; + public static final String ADLS_SHARED_KEY_ACCOUNT_KEY = "adls.auth.shared-key.account.key"; private Map adlsSasTokens = Collections.emptyMap(); private Map adlsConnectionStrings = Collections.emptyMap(); + private StorageSharedKeyCredential namedKeyCreds; private Integer adlsReadBlockSize; private Long adlsWriteBlockSize; @@ -44,6 +49,18 @@ public AzureProperties(Map properties) { this.adlsConnectionStrings = PropertyUtil.propertiesWithPrefix(properties, ADLS_CONNECTION_STRING_PREFIX); + String sharedKeyAccountName = properties.get(ADLS_SHARED_KEY_ACCOUNT_NAME); + String sharedKeyAccountKey = properties.get(ADLS_SHARED_KEY_ACCOUNT_KEY); + if (sharedKeyAccountName != null || sharedKeyAccountKey != null) { + Preconditions.checkArgument( + sharedKeyAccountName != null && sharedKeyAccountKey != null, + "Azure authentication: shared-key requires both %s and %s", + ADLS_SHARED_KEY_ACCOUNT_NAME, + ADLS_SHARED_KEY_ACCOUNT_KEY); + this.namedKeyCreds = + new StorageSharedKeyCredential(sharedKeyAccountName, sharedKeyAccountKey); + } + if (properties.containsKey(ADLS_READ_BLOCK_SIZE)) { this.adlsReadBlockSize = Integer.parseInt(properties.get(ADLS_READ_BLOCK_SIZE)); } @@ -64,6 +81,8 @@ public void applyClientConfiguration(String account, DataLakeFileSystemClientBui String sasToken = adlsSasTokens.get(account); if (sasToken != null && !sasToken.isEmpty()) { builder.sasToken(sasToken); + } else if (namedKeyCreds != null) { + builder.credential(namedKeyCreds); } else { builder.credential(new DefaultAzureCredentialBuilder().build()); } diff --git a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java index 3bc7815e877c..2ebccb625898 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -18,14 +18,19 @@ */ package org.apache.iceberg.azure; +import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_KEY; +import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_NAME; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import com.azure.core.credential.TokenCredential; +import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class AzurePropertiesTest { @@ -39,6 +44,7 @@ public void testWithSasToken() { props.applyClientConfiguration("account1", clientBuilder); verify(clientBuilder).sasToken(any()); verify(clientBuilder, times(0)).credential(any(TokenCredential.class)); + verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); } @Test @@ -50,6 +56,7 @@ public void testNoMatchingSasToken() { props.applyClientConfiguration("account2", clientBuilder); verify(clientBuilder, times(0)).sasToken(any()); verify(clientBuilder).credential(any(TokenCredential.class)); + verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); } @Test @@ -60,6 +67,7 @@ public void testNoSasToken() { props.applyClientConfiguration("account", clientBuilder); verify(clientBuilder, times(0)).sasToken(any()); verify(clientBuilder).credential(any(TokenCredential.class)); + verify(clientBuilder, never()).credential(any(StorageSharedKeyCredential.class)); } @Test @@ -90,4 +98,37 @@ public void testNoConnectionString() { props.applyClientConfiguration("account", clientBuilder); verify(clientBuilder).endpoint("https://account"); } + + @Test + public void testSharedKey() { + Assertions.assertThatIllegalArgumentException() + .isThrownBy( + () -> + new AzureProperties( + ImmutableMap.of( + ADLS_SHARED_KEY_ACCOUNT_KEY, "not-really-base64-encoded-key-here"))) + .withMessage( + String.format( + "Azure authentication: shared-key requires both %s and %s", + ADLS_SHARED_KEY_ACCOUNT_NAME, ADLS_SHARED_KEY_ACCOUNT_KEY)); + Assertions.assertThatIllegalArgumentException() + .isThrownBy( + () -> new AzureProperties(ImmutableMap.of(ADLS_SHARED_KEY_ACCOUNT_NAME, "account"))) + .withMessage( + String.format( + "Azure authentication: shared-key requires both %s and %s", + ADLS_SHARED_KEY_ACCOUNT_NAME, ADLS_SHARED_KEY_ACCOUNT_KEY)); + + AzureProperties props = + new AzureProperties( + ImmutableMap.of( + ADLS_SHARED_KEY_ACCOUNT_NAME, + "account", + ADLS_SHARED_KEY_ACCOUNT_KEY, + "not-really-base64-encoded-key-here")); + DataLakeFileSystemClientBuilder clientBuilder = mock(DataLakeFileSystemClientBuilder.class); + props.applyClientConfiguration("account", clientBuilder); + verify(clientBuilder).credential(any(StorageSharedKeyCredential.class)); + verify(clientBuilder, never()).credential(any(TokenCredential.class)); + } } From ccaeb2f4d5be44b4ab177b8fb790194eff8070a5 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 16 Nov 2023 08:33:02 -0800 Subject: [PATCH 37/94] Core: Disallow setting equality field IDs for data (#8970) --- .../java/org/apache/iceberg/ContentFileParser.java | 1 - core/src/main/java/org/apache/iceberg/DataFiles.java | 11 +++-------- .../main/java/org/apache/iceberg/GenericDataFile.java | 3 +-- .../org/apache/iceberg/TestContentFileParser.java | 6 ++---- .../apache/iceberg/TestManifestWriterVersions.java | 2 +- 5 files changed, 7 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ContentFileParser.java b/core/src/main/java/org/apache/iceberg/ContentFileParser.java index b3edf2927fbc..dd08c5c69e7d 100644 --- a/core/src/main/java/org/apache/iceberg/ContentFileParser.java +++ b/core/src/main/java/org/apache/iceberg/ContentFileParser.java @@ -156,7 +156,6 @@ static ContentFile fromJson(JsonNode jsonNode, PartitionSpec spec) { metrics, keyMetadata, splitOffsets, - equalityFieldIds, sortOrderId); } else { return new GenericDeleteFile( diff --git a/core/src/main/java/org/apache/iceberg/DataFiles.java b/core/src/main/java/org/apache/iceberg/DataFiles.java index a6ea80d3662a..34f830be4daa 100644 --- a/core/src/main/java/org/apache/iceberg/DataFiles.java +++ b/core/src/main/java/org/apache/iceberg/DataFiles.java @@ -29,7 +29,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Conversions; -import org.apache.iceberg.util.ArrayUtil; import org.apache.iceberg.util.ByteBuffers; public class DataFiles { @@ -154,7 +153,6 @@ public static class Builder { private Map upperBounds = null; private ByteBuffer keyMetadata = null; private List splitOffsets = null; - private List equalityFieldIds = null; private Integer sortOrderId = SortOrder.unsorted().orderId(); public Builder(PartitionSpec spec) { @@ -301,12 +299,10 @@ public Builder withSplitOffsets(List offsets) { return this; } + /** @deprecated since 1.5.0, will be removed in 1.6.0; must not be set for data files. */ + @Deprecated public Builder withEqualityFieldIds(List equalityIds) { - if (equalityIds != null) { - this.equalityFieldIds = ImmutableList.copyOf(equalityIds); - } - - return this; + throw new UnsupportedOperationException("Equality field IDs must not be set for data files"); } public Builder withEncryptionKeyMetadata(ByteBuffer newKeyMetadata) { @@ -350,7 +346,6 @@ public DataFile build() { upperBounds), keyMetadata, splitOffsets, - ArrayUtil.toIntArray(equalityFieldIds), sortOrderId); } } diff --git a/core/src/main/java/org/apache/iceberg/GenericDataFile.java b/core/src/main/java/org/apache/iceberg/GenericDataFile.java index f78bb2ce9f2e..8fe7ec756abf 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDataFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java @@ -41,7 +41,6 @@ class GenericDataFile extends BaseFile implements DataFile { Metrics metrics, ByteBuffer keyMetadata, List splitOffsets, - int[] equalityFieldIds, Integer sortOrderId) { super( specId, @@ -58,7 +57,7 @@ class GenericDataFile extends BaseFile implements DataFile { metrics.lowerBounds(), metrics.upperBounds(), splitOffsets, - equalityFieldIds, + null /* no equality field IDs */, sortOrderId, keyMetadata); } diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index 9360f571c5bb..f3b5e7b60c31 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.databind.JsonNode; import java.nio.ByteBuffer; import java.util.Arrays; -import java.util.Collections; import java.util.stream.Stream; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Comparators; @@ -141,7 +140,7 @@ private static String dataFileJsonWithAllOptional(PartitionSpec spec) { + "\"lower-bounds\":{\"keys\":[3,4],\"values\":[\"01000000\",\"02000000\"]}," + "\"upper-bounds\":{\"keys\":[3,4],\"values\":[\"05000000\",\"0A000000\"]}," + "\"key-metadata\":\"00000000000000000000000000000000\"," - + "\"split-offsets\":[128,256],\"equality-ids\":[1],\"sort-order-id\":1}"; + + "\"split-offsets\":[128,256],\"sort-order-id\":1}"; } else { return "{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-with-stats.parquet\"," + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":1},\"file-size-in-bytes\":350,\"record-count\":10," @@ -152,7 +151,7 @@ private static String dataFileJsonWithAllOptional(PartitionSpec spec) { + "\"lower-bounds\":{\"keys\":[3,4],\"values\":[\"01000000\",\"02000000\"]}," + "\"upper-bounds\":{\"keys\":[3,4],\"values\":[\"05000000\",\"0A000000\"]}," + "\"key-metadata\":\"00000000000000000000000000000000\"," - + "\"split-offsets\":[128,256],\"equality-ids\":[1],\"sort-order-id\":1}"; + + "\"split-offsets\":[128,256],\"sort-order-id\":1}"; } } @@ -180,7 +179,6 @@ private static DataFile dataFileWithAllOptional(PartitionSpec spec) { )) .withFileSizeInBytes(350) .withSplitOffsets(Arrays.asList(128L, 256L)) - .withEqualityFieldIds(Collections.singletonList(1)) .withEncryptionKeyMetadata(ByteBuffer.wrap(new byte[16])) .withSortOrder( SortOrder.builderFor(TableTestBase.SCHEMA) diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 08b27d7460da..740791b255d5 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -78,7 +78,7 @@ public class TestManifestWriterVersions { private static final DataFile DATA_FILE = new GenericDataFile( - 0, PATH, FORMAT, PARTITION, 150972L, METRICS, null, OFFSETS, null, SORT_ORDER_ID); + 0, PATH, FORMAT, PARTITION, 150972L, METRICS, null, OFFSETS, SORT_ORDER_ID); private static final List EQUALITY_IDS = ImmutableList.of(1); private static final int[] EQUALITY_ID_ARR = new int[] {1}; From 2e2ac8dfd2ab2c214444cccb0e6024e1f80b0502 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 16 Nov 2023 09:31:17 -0800 Subject: [PATCH 38/94] Core: Fix split size calculations in file rewriters (#9069) --- .../actions/SizeBasedFileRewriter.java | 16 ++- .../actions/TestSizeBasedRewriter.java | 100 ++++++++++++++++++ 2 files changed, 112 insertions(+), 4 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java diff --git a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java index 505fbaeda89d..cf98c5266ae0 100644 --- a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java +++ b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java @@ -181,13 +181,21 @@ protected long inputSize(List group) { } /** - * Returns the smallest of our max write file threshold and our estimated split size based on the - * number of output files we want to generate. Add an overhead onto the estimated split size to - * try to avoid small errors in size creating brand-new files. + * Calculates the split size to use in bin-packing rewrites. + * + *

This method determines the target split size as the input size divided by the desired number + * of output files. The final split size is adjusted to be at least as big as the target file size + * but less than the max write file size. */ protected long splitSize(long inputSize) { long estimatedSplitSize = (inputSize / numOutputFiles(inputSize)) + SPLIT_OVERHEAD; - return Math.min(estimatedSplitSize, writeMaxFileSize()); + if (estimatedSplitSize < targetFileSize) { + return targetFileSize; + } else if (estimatedSplitSize > writeMaxFileSize()) { + return writeMaxFileSize(); + } else { + return estimatedSplitSize; + } } /** diff --git a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java new file mode 100644 index 000000000000..c33bbc6f6d1e --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.actions; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MockFileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestSizeBasedRewriter extends TableTestBase { + + @Parameterized.Parameters(name = "formatVersion = {0}") + public static Object[] parameters() { + return new Object[] {1, 2}; + } + + public TestSizeBasedRewriter(int formatVersion) { + super(formatVersion); + } + + @Test + public void testSplitSizeLowerBound() { + SizeBasedDataFileRewriterImpl rewriter = new SizeBasedDataFileRewriterImpl(table); + + FileScanTask task1 = new MockFileScanTask(145L * 1024 * 1024); + FileScanTask task2 = new MockFileScanTask(145L * 1024 * 1024); + FileScanTask task3 = new MockFileScanTask(145L * 1024 * 1024); + FileScanTask task4 = new MockFileScanTask(145L * 1024 * 1024); + List tasks = ImmutableList.of(task1, task2, task3, task4); + + long minFileSize = 256L * 1024 * 1024; + long targetFileSize = 512L * 1024 * 1024; + long maxFileSize = 768L * 1024 * 1024; + + Map options = + ImmutableMap.of( + SizeBasedDataRewriter.MIN_FILE_SIZE_BYTES, String.valueOf(minFileSize), + SizeBasedDataRewriter.TARGET_FILE_SIZE_BYTES, String.valueOf(targetFileSize), + SizeBasedDataRewriter.MAX_FILE_SIZE_BYTES, String.valueOf(maxFileSize)); + rewriter.init(options); + + // the total task size is 580 MB and the target file size is 512 MB + // the remainder must be written into a separate file as it exceeds 10% + long numOutputFiles = rewriter.computeNumOutputFiles(tasks); + assertThat(numOutputFiles).isEqualTo(2); + + // the split size must be >= targetFileSize and < maxFileSize + long splitSize = rewriter.computeSplitSize(tasks); + assertThat(splitSize).isGreaterThanOrEqualTo(targetFileSize); + assertThat(splitSize).isLessThan(maxFileSize); + } + + private static class SizeBasedDataFileRewriterImpl extends SizeBasedDataRewriter { + + SizeBasedDataFileRewriterImpl(Table table) { + super(table); + } + + @Override + public Set rewrite(List group) { + throw new UnsupportedOperationException("Not implemented"); + } + + public long computeSplitSize(List group) { + return splitSize(inputSize(group)); + } + + public long computeNumOutputFiles(List group) { + return numOutputFiles(inputSize(group)); + } + } +} From 3d6072ad4d94ba8fb83fcecfdaa2a2a326e99e15 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 16 Nov 2023 15:41:29 -0800 Subject: [PATCH 39/94] API: Add CharSequenceMap (#9047) --- .../apache/iceberg/util/CharSequenceMap.java | 220 ++++++++++++++++++ .../iceberg/util/TestCharSequenceMap.java | 202 ++++++++++++++++ 2 files changed, 422 insertions(+) create mode 100644 api/src/main/java/org/apache/iceberg/util/CharSequenceMap.java create mode 100644 api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java diff --git a/api/src/main/java/org/apache/iceberg/util/CharSequenceMap.java b/api/src/main/java/org/apache/iceberg/util/CharSequenceMap.java new file mode 100644 index 000000000000..98da48ed590c --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/util/CharSequenceMap.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; + +/** + * A map that uses char sequences as keys. + * + *

This implementation wraps provided keys into {@link CharSequenceWrapper} for consistent + * hashing and equals behavior. This ensures that objects of different types that represent the same + * sequence of characters are treated as equal keys in the map. + * + *

Note: This map is not designed for concurrent modification by multiple threads. However, it + * supports safe concurrent reads, assuming there are no concurrent writes. + * + *

Note: This map does not support null keys. + * + * @param the type of values + */ +public class CharSequenceMap implements Map, Serializable { + + private static final long serialVersionUID = 1L; + private static final ThreadLocal WRAPPERS = + ThreadLocal.withInitial(() -> CharSequenceWrapper.wrap(null)); + + private final Map wrapperMap; + + private CharSequenceMap() { + this.wrapperMap = Maps.newHashMap(); + } + + public static CharSequenceMap create() { + return new CharSequenceMap<>(); + } + + @Override + public int size() { + return wrapperMap.size(); + } + + @Override + public boolean isEmpty() { + return wrapperMap.isEmpty(); + } + + @Override + public boolean containsKey(Object key) { + if (key instanceof CharSequence) { + CharSequenceWrapper wrapper = WRAPPERS.get(); + boolean result = wrapperMap.containsKey(wrapper.set((CharSequence) key)); + wrapper.set(null); // don't hold a reference to the key + return result; + } + + return false; + } + + @Override + public boolean containsValue(Object value) { + return wrapperMap.containsValue(value); + } + + @Override + public V get(Object key) { + if (key instanceof CharSequence) { + CharSequenceWrapper wrapper = WRAPPERS.get(); + V result = wrapperMap.get(wrapper.set((CharSequence) key)); + wrapper.set(null); // don't hold a reference to the value + return result; + } + + return null; + } + + @Override + public V put(CharSequence key, V value) { + return wrapperMap.put(CharSequenceWrapper.wrap(key), value); + } + + @Override + public V remove(Object key) { + if (key instanceof CharSequence) { + CharSequenceWrapper wrapper = WRAPPERS.get(); + V result = wrapperMap.remove(wrapper.set((CharSequence) key)); + wrapper.set(null); // don't hold a reference to the value + return result; + } + + return null; + } + + @Override + public void putAll(Map otherMap) { + otherMap.forEach(this::put); + } + + @Override + public void clear() { + wrapperMap.clear(); + } + + @Override + public Set keySet() { + CharSequenceSet keySet = CharSequenceSet.empty(); + + for (CharSequenceWrapper wrapper : wrapperMap.keySet()) { + keySet.add(wrapper.get()); + } + + return keySet; + } + + @Override + public Collection values() { + return wrapperMap.values(); + } + + @Override + public Set> entrySet() { + Set> entrySet = Sets.newHashSet(); + + for (Entry entry : wrapperMap.entrySet()) { + entrySet.add(new CharSequenceEntry<>(entry)); + } + + return entrySet; + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + CharSequenceMap that = (CharSequenceMap) other; + return Objects.equals(wrapperMap, that.wrapperMap); + } + + @Override + public int hashCode() { + return Objects.hashCode(wrapperMap); + } + + @Override + public String toString() { + return entrySet().stream().map(this::toString).collect(Collectors.joining(", ", "{", "}")); + } + + private String toString(Entry entry) { + CharSequence key = entry.getKey(); + V value = entry.getValue(); + return key + "=" + (value == this ? "(this Map)" : value); + } + + private static class CharSequenceEntry implements Entry { + private final Entry inner; + + private CharSequenceEntry(Entry inner) { + this.inner = inner; + } + + @Override + public CharSequence getKey() { + return inner.getKey().get(); + } + + @Override + public V getValue() { + return inner.getValue(); + } + + @Override + public int hashCode() { + return inner.hashCode(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + CharSequenceEntry that = (CharSequenceEntry) other; + return inner.equals(that.inner); + } + + @Override + public V setValue(V value) { + throw new UnsupportedOperationException("Cannot set value"); + } + } +} diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java new file mode 100644 index 000000000000..47d686d3abbe --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java @@ -0,0 +1,202 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.util; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +public class TestCharSequenceMap { + + @Test + public void testEmptyMap() { + CharSequenceMap map = CharSequenceMap.create(); + assertThat(map).isEmpty(); + assertThat(map).hasSize(0); + assertThat(map).doesNotContainKey("key"); + assertThat(map).doesNotContainValue("value"); + assertThat(map.values()).isEmpty(); + assertThat(map.keySet()).isEmpty(); + assertThat(map.entrySet()).isEmpty(); + } + + @Test + public void testDifferentCharSequenceImplementations() { + CharSequenceMap map = CharSequenceMap.create(); + map.put("abc", "value1"); + map.put(new StringBuffer("def"), "value2"); + assertThat(map).containsEntry(new StringBuilder("abc"), "value1"); + assertThat(map).containsEntry("def", "value2"); + } + + @Test + public void testPutAndGet() { + CharSequenceMap map = CharSequenceMap.create(); + map.put("key1", "value1"); + assertThat(map).containsEntry("key1", "value1"); + } + + @Test + public void testRemove() { + CharSequenceMap map = CharSequenceMap.create(); + map.put("key1", "value1"); + map.remove(new StringBuilder("key1")); + assertThat(map).doesNotContainKey("key1"); + assertThat(map).isEmpty(); + } + + @Test + public void testPutAll() { + CharSequenceMap map = CharSequenceMap.create(); + map.putAll(ImmutableMap.of("key1", "value1", "key2", "value2")); + assertThat(map).containsEntry("key1", "value1"); + assertThat(map).containsEntry("key2", "value2"); + } + + @Test + public void testClear() { + CharSequenceMap map = CharSequenceMap.create(); + map.put("key1", "value1"); + map.clear(); + assertThat(map).isEmpty(); + } + + @Test + public void testValues() { + CharSequenceMap map = CharSequenceMap.create(); + map.put("key1", "value1"); + map.put("key2", "value2"); + assertThat(map.values()).containsAll(ImmutableList.of("value1", "value2")); + } + + @Test + public void testEntrySet() { + CharSequenceMap map = CharSequenceMap.create(); + map.put("key1", "value1"); + map.put(new StringBuilder("key2"), "value2"); + assertThat(map.entrySet()).hasSize(2); + } + + @Test + public void testEquals() { + CharSequenceMap map1 = CharSequenceMap.create(); + map1.put(new StringBuilder("key"), "value"); + + CharSequenceMap map2 = CharSequenceMap.create(); + map2.put("key", "value"); + + assertThat(map1).isEqualTo(map2); + } + + @Test + public void testHashCode() { + CharSequenceMap map1 = CharSequenceMap.create(); + map1.put(new StringBuilder("key"), "value"); + + CharSequenceMap map2 = CharSequenceMap.create(); + map2.put("key", "value"); + + assertThat(map1.hashCode()).isEqualTo(map2.hashCode()); + } + + @Test + public void testToString() { + CharSequenceMap map = CharSequenceMap.create(); + + // empty map + assertThat(map.toString()).isEqualTo("{}"); + + // single entry + map.put("key1", "value1"); + assertThat(map.toString()).isEqualTo("{key1=value1}"); + + // multiple entries + map.put("key2", "value2"); + map.put("key3", "value3"); + String toStringResult = map.toString(); + assertThat(toStringResult).contains("key1=value1", "key2=value2", "key3=value3"); + } + + @Test + public void testComputeIfAbsent() { + CharSequenceMap map = CharSequenceMap.create(); + + String result1 = map.computeIfAbsent("key1", key -> "computedValue1"); + assertThat(result1).isEqualTo("computedValue1"); + assertThat(map).containsEntry("key1", "computedValue1"); + + // verify existing key is not affected + String result2 = map.computeIfAbsent("key1", key -> "newValue"); + assertThat(result2).isEqualTo("computedValue1"); + assertThat(map).containsEntry("key1", "computedValue1"); + } + + @Test + public void testMerge() { + CharSequenceMap map = CharSequenceMap.create(); + map.put("key1", "value1"); + map.put("key2", "value2"); + + // merge with an existing key + map.merge(new StringBuilder("key1"), "newValue", (oldVal, newVal) -> oldVal + newVal); + assertThat(map).containsEntry("key1", "value1newValue"); + + // merge with a non-existing key + map.merge(new StringBuffer("key3"), "value3", (oldVal, newVal) -> oldVal + newVal); + assertThat(map).containsEntry("key3", "value3"); + + // merge with null BiFunction should replace the value + map.merge("key2", "replacedValue", (oldVal, newVal) -> null); + assertThat(map).doesNotContainKey("key2"); + + // merge when old value is null (should add new value) + map.remove("key1"); + map.merge("key1", "reAddedValue", (oldVal, newVal) -> oldVal + newVal); + assertThat(map).containsEntry("key1", "reAddedValue"); + } + + @Test + public void testConcurrentReadAccess() throws InterruptedException { + CharSequenceMap map = CharSequenceMap.create(); + map.put("key1", "value1"); + map.put("key2", "value2"); + map.put("key3", "value3"); + + int numThreads = 10; + ExecutorService executorService = Executors.newFixedThreadPool(numThreads); + + // read the map from multiple threads to ensure thread-local wrappers are used + for (int i = 0; i < numThreads; i++) { + executorService.submit( + () -> { + assertThat(map.get("key1")).isEqualTo("value1"); + assertThat(map.get("key2")).isEqualTo("value2"); + assertThat(map.get("key3")).isEqualTo("value3"); + }); + } + + executorService.shutdown(); + assertThat(executorService.awaitTermination(1, TimeUnit.MINUTES)).isTrue(); + } +} From 1e2a71398fb5564d1cd9f4e12b3d1acc568f8ef7 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 16 Nov 2023 16:16:15 -0800 Subject: [PATCH 40/94] Parquet: Add log entry when Bloom filters are used (#9010) Co-authored-by: Huaxin Gao --- .../parquet/ParquetBloomRowGroupFilter.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java index de7f46a01851..11201bd88e5a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java @@ -48,8 +48,13 @@ import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ParquetBloomRowGroupFilter { + + private static final Logger LOG = LoggerFactory.getLogger(ParquetBloomRowGroupFilter.class); + private final Schema schema; private final Expression expr; private final boolean caseSensitive; @@ -114,10 +119,13 @@ private boolean eval( Set filterRefs = Binder.boundReferences(schema.asStruct(), ImmutableList.of(expr), caseSensitive); - // If the filter's column set doesn't overlap with any bloom filter columns, exit early with - // ROWS_MIGHT_MATCH - if (!filterRefs.isEmpty() && Sets.intersection(fieldsWithBloomFilter, filterRefs).isEmpty()) { - return ROWS_MIGHT_MATCH; + if (!filterRefs.isEmpty()) { + Set overlappedBloomFilters = Sets.intersection(fieldsWithBloomFilter, filterRefs); + if (overlappedBloomFilters.isEmpty()) { + return ROWS_MIGHT_MATCH; + } else { + LOG.debug("Using Bloom filters for columns with IDs: {}", overlappedBloomFilters); + } } return ExpressionVisitors.visitEvaluator(expr, this); From 17c7815d433ae075e16351f264ecf17598169dc9 Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Fri, 17 Nov 2023 14:52:17 +0100 Subject: [PATCH 41/94] GCS: Allow no-auth for testing purposes (#9061) Although there is no "official" Google Cloud Storage emulator available yet, there is [one available](https://github.com/oittaa/gcp-storage-emulator) that allows at least some basic testing. To use an emulator, the client needs to be configured to use no authentication, otherwise it will fallback to "automatic credential detection". --- .../org/apache/iceberg/gcp/GCPProperties.java | 14 ++++++ .../org/apache/iceberg/gcp/gcs/GCSFileIO.java | 9 ++++ .../apache/iceberg/gcp/GCPPropertiesTest.java | 50 +++++++++++++++++++ 3 files changed, 73 insertions(+) create mode 100644 gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java index 55a8fdcfee34..4465ee29012a 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -22,6 +22,7 @@ import java.util.Date; import java.util.Map; import java.util.Optional; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.PropertyUtil; public class GCPProperties implements Serializable { @@ -40,6 +41,8 @@ public class GCPProperties implements Serializable { public static final String GCS_OAUTH2_TOKEN = "gcs.oauth2.token"; public static final String GCS_OAUTH2_TOKEN_EXPIRES_AT = "gcs.oauth2.token-expires-at"; + // Boolean to explicitly configure "no authentication" for testing purposes using a GCS emulator + public static final String GCS_NO_AUTH = "gcs.no-auth"; /** Configure the batch size used when deleting multiple files from a given GCS bucket */ public static final String GCS_DELETE_BATCH_SIZE = "gcs.delete.batch-size"; @@ -60,6 +63,7 @@ public class GCPProperties implements Serializable { private Integer gcsChannelReadChunkSize; private Integer gcsChannelWriteChunkSize; + private boolean gcsNoAuth; private String gcsOAuth2Token; private Date gcsOAuth2TokenExpiresAt; @@ -90,6 +94,12 @@ public GCPProperties(Map properties) { gcsOAuth2TokenExpiresAt = new Date(Long.parseLong(properties.get(GCS_OAUTH2_TOKEN_EXPIRES_AT))); } + gcsNoAuth = Boolean.parseBoolean(properties.getOrDefault(GCS_NO_AUTH, "false")); + Preconditions.checkState( + !(gcsOAuth2Token != null && gcsNoAuth), + "Invalid auth settings: must not configure %s and %s", + GCS_NO_AUTH, + GCS_OAUTH2_TOKEN); gcsDeleteBatchSize = PropertyUtil.propertyAsInt( @@ -132,6 +142,10 @@ public Optional oauth2Token() { return Optional.ofNullable(gcsOAuth2Token); } + public boolean noAuth() { + return gcsNoAuth; + } + public Optional oauth2TokenExpiresAt() { return Optional.ofNullable(gcsOAuth2TokenExpiresAt); } diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java index 09eb4a74008d..2201c876bd38 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSFileIO.java @@ -20,6 +20,7 @@ import com.google.auth.oauth2.AccessToken; import com.google.auth.oauth2.OAuth2Credentials; +import com.google.cloud.NoCredentials; import com.google.cloud.storage.Blob; import com.google.cloud.storage.BlobId; import com.google.cloud.storage.Storage; @@ -141,10 +142,18 @@ public void initialize(Map props) { gcpProperties.clientLibToken().ifPresent(builder::setClientLibToken); gcpProperties.serviceHost().ifPresent(builder::setHost); + // Google Cloud APIs default to automatically detect the credentials to use, which is + // in most cases the convenient way, especially in GCP. + // See javadoc of com.google.auth.oauth2.GoogleCredentials.getApplicationDefault(). + if (gcpProperties.noAuth()) { + // Explicitly allow "no credentials" for testing purposes. + builder.setCredentials(NoCredentials.getInstance()); + } gcpProperties .oauth2Token() .ifPresent( token -> { + // Explicitly configure an OAuth token. AccessToken accessToken = new AccessToken(token, gcpProperties.oauth2TokenExpiresAt().orElse(null)); builder.setCredentials(OAuth2Credentials.create(accessToken)); diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java b/gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java new file mode 100644 index 000000000000..f7c770dbb583 --- /dev/null +++ b/gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.gcp; + +import static org.apache.iceberg.gcp.GCPProperties.GCS_NO_AUTH; +import static org.apache.iceberg.gcp.GCPProperties.GCS_OAUTH2_TOKEN; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class GCPPropertiesTest { + + @Test + public void testOAuthWithNoAuth() { + Assertions.assertThatIllegalStateException() + .isThrownBy( + () -> + new GCPProperties(ImmutableMap.of(GCS_OAUTH2_TOKEN, "oauth", GCS_NO_AUTH, "true"))) + .withMessage( + String.format( + "Invalid auth settings: must not configure %s and %s", + GCS_NO_AUTH, GCS_OAUTH2_TOKEN)); + + GCPProperties gcpProperties = + new GCPProperties(ImmutableMap.of(GCS_OAUTH2_TOKEN, "oauth", GCS_NO_AUTH, "false")); + assertThat(gcpProperties.noAuth()).isFalse(); + assertThat(gcpProperties.oauth2Token()).get().isEqualTo("oauth"); + gcpProperties = new GCPProperties(ImmutableMap.of(GCS_NO_AUTH, "true")); + assertThat(gcpProperties.noAuth()).isTrue(); + assertThat(gcpProperties.oauth2Token()).isNotPresent(); + } +} From abbfdae108dd342191dcc685021edbc0113fe302 Mon Sep 17 00:00:00 2001 From: zhen Date: Sat, 18 Nov 2023 09:26:32 +0800 Subject: [PATCH 42/94] Spark 3.4, 3.3: Support metadata columns in staged scans (#9098) This change cherry-picks PR #8872 to Spark 3.4 and 3.3. --- ...TestMetaColumnProjectionWithStageScan.java | 127 ++++++++++++++++++ .../iceberg/spark/source/SparkStagedScan.java | 10 +- .../spark/source/SparkStagedScanBuilder.java | 52 ++++++- ...TestMetaColumnProjectionWithStageScan.java | 127 ++++++++++++++++++ .../iceberg/spark/source/SparkStagedScan.java | 10 +- .../spark/source/SparkStagedScanBuilder.java | 52 ++++++- 6 files changed, 368 insertions(+), 10 deletions(-) create mode 100644 spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java create mode 100644 spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java new file mode 100644 index 000000000000..e9013848cf11 --- /dev/null +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Test; +import org.junit.runners.Parameterized; + +public class TestMetaColumnProjectionWithStageScan extends SparkExtensionsTestBase { + + public TestMetaColumnProjectionWithStageScan( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties() + } + }; + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + private void stageTask( + Table tab, String fileSetID, CloseableIterable tasks) { + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + taskSetManager.stageTasks(tab, fileSetID, Lists.newArrayList(tasks)); + } + + @Test + public void testReadStageTableMeta() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" + + "('format-version'='2', 'write.delete.mode'='merge-on-read')", + tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + table.refresh(); + String tableLocation = table.location(); + + try (CloseableIterable tasks = table.newBatchScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(table, fileSetID, tasks); + Dataset scanDF2 = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .load(tableLocation); + + Assertions.assertThat(scanDF2.columns().length).isEqualTo(2); + } + + try (CloseableIterable tasks = table.newBatchScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(table, fileSetID, tasks); + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .load(tableLocation) + .select("*", "_pos"); + + List rows = scanDF.collectAsList(); + ImmutableList expectedRows = + ImmutableList.of(row(1L, "a", 0L), row(2L, "b", 1L), row(3L, "c", 2L), row(4L, "d", 3L)); + assertEquals("result should match", expectedRows, rowsToJava(rows)); + } + } +} diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java index 89b184c91c51..ad501f7f91d6 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java @@ -22,6 +22,7 @@ import java.util.Objects; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -40,8 +41,11 @@ class SparkStagedScan extends SparkScan { private List> taskGroups = null; // lazy cache of tasks SparkStagedScan(SparkSession spark, Table table, SparkReadConf readConf) { - super(spark, table, readConf, table.schema(), ImmutableList.of()); + this(spark, table, table.schema(), readConf); + } + SparkStagedScan(SparkSession spark, Table table, Schema expectedSchema, SparkReadConf readConf) { + super(spark, table, readConf, expectedSchema, ImmutableList.of()); this.taskSetId = readConf.scanTaskSetId(); this.splitSize = readConf.splitSize(); this.splitLookback = readConf.splitLookback(); @@ -77,6 +81,7 @@ public boolean equals(Object other) { SparkStagedScan that = (SparkStagedScan) other; return table().name().equals(that.table().name()) && Objects.equals(taskSetId, that.taskSetId) + && readSchema().equals(that.readSchema()) && Objects.equals(splitSize, that.splitSize) && Objects.equals(splitLookback, that.splitLookback) && Objects.equals(openFileCost, that.openFileCost); @@ -84,7 +89,8 @@ public boolean equals(Object other) { @Override public int hashCode() { - return Objects.hash(table().name(), taskSetId, splitSize, splitSize, openFileCost); + return Objects.hash( + table().name(), taskSetId, readSchema(), splitSize, splitSize, openFileCost); } @Override diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index 37bbea42e5b1..25393888f95c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -18,27 +18,75 @@ */ package org.apache.iceberg.spark.source; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -class SparkStagedScanBuilder implements ScanBuilder { +class SparkStagedScanBuilder implements ScanBuilder, SupportsPushDownRequiredColumns { private final SparkSession spark; private final Table table; private final SparkReadConf readConf; + private final List metaColumns = Lists.newArrayList(); + + private Schema schema = null; SparkStagedScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { this.spark = spark; this.table = table; this.readConf = new SparkReadConf(spark, table, options); + this.schema = table.schema(); } @Override public Scan build() { - return new SparkStagedScan(spark, table, readConf); + return new SparkStagedScan(spark, table, schemaWithMetadataColumns(), readConf); + } + + @Override + public void pruneColumns(StructType requestedSchema) { + StructType requestedProjection = removeMetaColumns(requestedSchema); + this.schema = SparkSchemaUtil.prune(schema, requestedProjection); + + Stream.of(requestedSchema.fields()) + .map(StructField::name) + .filter(MetadataColumns::isMetadataColumn) + .distinct() + .forEach(metaColumns::add); + } + + private StructType removeMetaColumns(StructType structType) { + return new StructType( + Stream.of(structType.fields()) + .filter(field -> MetadataColumns.nonMetadataColumn(field.name())) + .toArray(StructField[]::new)); + } + + private Schema schemaWithMetadataColumns() { + // metadata columns + List fields = + metaColumns.stream() + .distinct() + .map(name -> MetadataColumns.metadataColumn(table, name)) + .collect(Collectors.toList()); + Schema meta = new Schema(fields); + + // schema of rows returned by readers + return TypeUtil.join(schema, meta); } } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java new file mode 100644 index 000000000000..e9013848cf11 --- /dev/null +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.extensions; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.ScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.ScanTaskSetManager; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Test; +import org.junit.runners.Parameterized; + +public class TestMetaColumnProjectionWithStageScan extends SparkExtensionsTestBase { + + public TestMetaColumnProjectionWithStageScan( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties() + } + }; + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + } + + private void stageTask( + Table tab, String fileSetID, CloseableIterable tasks) { + ScanTaskSetManager taskSetManager = ScanTaskSetManager.get(); + taskSetManager.stageTasks(tab, fileSetID, Lists.newArrayList(tasks)); + } + + @Test + public void testReadStageTableMeta() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" + + "('format-version'='2', 'write.delete.mode'='merge-on-read')", + tableName); + + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + table.refresh(); + String tableLocation = table.location(); + + try (CloseableIterable tasks = table.newBatchScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(table, fileSetID, tasks); + Dataset scanDF2 = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .load(tableLocation); + + Assertions.assertThat(scanDF2.columns().length).isEqualTo(2); + } + + try (CloseableIterable tasks = table.newBatchScan().planFiles()) { + String fileSetID = UUID.randomUUID().toString(); + stageTask(table, fileSetID, tasks); + Dataset scanDF = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.FILE_OPEN_COST, "0") + .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) + .load(tableLocation) + .select("*", "_pos"); + + List rows = scanDF.collectAsList(); + ImmutableList expectedRows = + ImmutableList.of(row(1L, "a", 0L), row(2L, "b", 1L), row(3L, "c", 2L), row(4L, "d", 3L)); + assertEquals("result should match", expectedRows, rowsToJava(rows)); + } + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java index 0290bf7e84ce..fd299ade7fdc 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScan.java @@ -22,6 +22,7 @@ import java.util.Objects; import org.apache.iceberg.ScanTask; import org.apache.iceberg.ScanTaskGroup; +import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -39,9 +40,8 @@ class SparkStagedScan extends SparkScan { private List> taskGroups = null; // lazy cache of tasks - SparkStagedScan(SparkSession spark, Table table, SparkReadConf readConf) { - super(spark, table, readConf, table.schema(), ImmutableList.of(), null); - + SparkStagedScan(SparkSession spark, Table table, Schema expectedSchema, SparkReadConf readConf) { + super(spark, table, readConf, expectedSchema, ImmutableList.of(), null); this.taskSetId = readConf.scanTaskSetId(); this.splitSize = readConf.splitSize(); this.splitLookback = readConf.splitLookback(); @@ -77,6 +77,7 @@ public boolean equals(Object other) { SparkStagedScan that = (SparkStagedScan) other; return table().name().equals(that.table().name()) && Objects.equals(taskSetId, that.taskSetId) + && readSchema().equals(that.readSchema()) && Objects.equals(splitSize, that.splitSize) && Objects.equals(splitLookback, that.splitLookback) && Objects.equals(openFileCost, that.openFileCost); @@ -84,7 +85,8 @@ public boolean equals(Object other) { @Override public int hashCode() { - return Objects.hash(table().name(), taskSetId, splitSize, splitSize, openFileCost); + return Objects.hash( + table().name(), taskSetId, readSchema(), splitSize, splitSize, openFileCost); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index 37bbea42e5b1..25393888f95c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -18,27 +18,75 @@ */ package org.apache.iceberg.spark.source; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkReadConf; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.connector.read.Scan; import org.apache.spark.sql.connector.read.ScanBuilder; +import org.apache.spark.sql.connector.read.SupportsPushDownRequiredColumns; +import org.apache.spark.sql.types.StructField; +import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -class SparkStagedScanBuilder implements ScanBuilder { +class SparkStagedScanBuilder implements ScanBuilder, SupportsPushDownRequiredColumns { private final SparkSession spark; private final Table table; private final SparkReadConf readConf; + private final List metaColumns = Lists.newArrayList(); + + private Schema schema = null; SparkStagedScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { this.spark = spark; this.table = table; this.readConf = new SparkReadConf(spark, table, options); + this.schema = table.schema(); } @Override public Scan build() { - return new SparkStagedScan(spark, table, readConf); + return new SparkStagedScan(spark, table, schemaWithMetadataColumns(), readConf); + } + + @Override + public void pruneColumns(StructType requestedSchema) { + StructType requestedProjection = removeMetaColumns(requestedSchema); + this.schema = SparkSchemaUtil.prune(schema, requestedProjection); + + Stream.of(requestedSchema.fields()) + .map(StructField::name) + .filter(MetadataColumns::isMetadataColumn) + .distinct() + .forEach(metaColumns::add); + } + + private StructType removeMetaColumns(StructType structType) { + return new StructType( + Stream.of(structType.fields()) + .filter(field -> MetadataColumns.nonMetadataColumn(field.name())) + .toArray(StructField[]::new)); + } + + private Schema schemaWithMetadataColumns() { + // metadata columns + List fields = + metaColumns.stream() + .distinct() + .map(name -> MetadataColumns.metadataColumn(table, name)) + .collect(Collectors.toList()); + Schema meta = new Schema(fields); + + // schema of rows returned by readers + return TypeUtil.join(schema, meta); } } From e69418ae8d6d041acae0af4fe59ea3c8b8f8705f Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 17 Nov 2023 18:34:25 -0800 Subject: [PATCH 43/94] Spark 3.5: Extend action for rewriting manifests to support deletes (#9020) --- .../iceberg/spark/SparkContentFile.java | 243 +++++++++++++ .../apache/iceberg/spark/SparkDataFile.java | 192 +---------- .../apache/iceberg/spark/SparkDeleteFile.java | 40 +++ .../actions/RewriteManifestsSparkAction.java | 237 ++++++++++--- .../actions/TestRewriteManifestsAction.java | 318 ++++++++++++++++++ .../spark/source/TestSparkDataFile.java | 194 ++++++++--- 6 files changed, 944 insertions(+), 280 deletions(-) create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java new file mode 100644 index 000000000000..3dd8049c13f6 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkContentFile.java @@ -0,0 +1,243 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.StructProjection; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; + +public abstract class SparkContentFile implements ContentFile { + + private final int fileContentPosition; + private final int filePathPosition; + private final int fileFormatPosition; + private final int partitionPosition; + private final int recordCountPosition; + private final int fileSizeInBytesPosition; + private final int columnSizesPosition; + private final int valueCountsPosition; + private final int nullValueCountsPosition; + private final int nanValueCountsPosition; + private final int lowerBoundsPosition; + private final int upperBoundsPosition; + private final int keyMetadataPosition; + private final int splitOffsetsPosition; + private final int sortOrderIdPosition; + private final int equalityIdsPosition; + private final Type lowerBoundsType; + private final Type upperBoundsType; + private final Type keyMetadataType; + + private final SparkStructLike wrappedPartition; + private final StructLike projectedPartition; + private Row wrapped; + + SparkContentFile(Types.StructType type, Types.StructType projectedType, StructType sparkType) { + this.lowerBoundsType = type.fieldType(DataFile.LOWER_BOUNDS.name()); + this.upperBoundsType = type.fieldType(DataFile.UPPER_BOUNDS.name()); + this.keyMetadataType = type.fieldType(DataFile.KEY_METADATA.name()); + + Types.StructType partitionType = type.fieldType(DataFile.PARTITION_NAME).asStructType(); + this.wrappedPartition = new SparkStructLike(partitionType); + + if (projectedType != null) { + Types.StructType projectedPartitionType = + projectedType.fieldType(DataFile.PARTITION_NAME).asStructType(); + StructProjection partitionProjection = + StructProjection.create(partitionType, projectedPartitionType); + this.projectedPartition = partitionProjection.wrap(wrappedPartition); + } else { + this.projectedPartition = wrappedPartition; + } + + Map positions = Maps.newHashMap(); + for (Types.NestedField field : type.fields()) { + String fieldName = field.name(); + positions.put(fieldName, fieldPosition(fieldName, sparkType)); + } + + this.fileContentPosition = positions.get(DataFile.CONTENT.name()); + this.filePathPosition = positions.get(DataFile.FILE_PATH.name()); + this.fileFormatPosition = positions.get(DataFile.FILE_FORMAT.name()); + this.partitionPosition = positions.get(DataFile.PARTITION_NAME); + this.recordCountPosition = positions.get(DataFile.RECORD_COUNT.name()); + this.fileSizeInBytesPosition = positions.get(DataFile.FILE_SIZE.name()); + this.columnSizesPosition = positions.get(DataFile.COLUMN_SIZES.name()); + this.valueCountsPosition = positions.get(DataFile.VALUE_COUNTS.name()); + this.nullValueCountsPosition = positions.get(DataFile.NULL_VALUE_COUNTS.name()); + this.nanValueCountsPosition = positions.get(DataFile.NAN_VALUE_COUNTS.name()); + this.lowerBoundsPosition = positions.get(DataFile.LOWER_BOUNDS.name()); + this.upperBoundsPosition = positions.get(DataFile.UPPER_BOUNDS.name()); + this.keyMetadataPosition = positions.get(DataFile.KEY_METADATA.name()); + this.splitOffsetsPosition = positions.get(DataFile.SPLIT_OFFSETS.name()); + this.sortOrderIdPosition = positions.get(DataFile.SORT_ORDER_ID.name()); + this.equalityIdsPosition = positions.get(DataFile.EQUALITY_IDS.name()); + } + + public F wrap(Row row) { + this.wrapped = row; + if (wrappedPartition.size() > 0) { + wrappedPartition.wrap(row.getAs(partitionPosition)); + } + return asFile(); + } + + protected abstract F asFile(); + + @Override + public Long pos() { + return null; + } + + @Override + public int specId() { + return -1; + } + + @Override + public FileContent content() { + if (wrapped.isNullAt(fileContentPosition)) { + return null; + } + return FileContent.values()[wrapped.getInt(fileContentPosition)]; + } + + @Override + public CharSequence path() { + return wrapped.getAs(filePathPosition); + } + + @Override + public FileFormat format() { + return FileFormat.fromString(wrapped.getString(fileFormatPosition)); + } + + @Override + public StructLike partition() { + return projectedPartition; + } + + @Override + public long recordCount() { + return wrapped.getAs(recordCountPosition); + } + + @Override + public long fileSizeInBytes() { + return wrapped.getAs(fileSizeInBytesPosition); + } + + @Override + public Map columnSizes() { + return wrapped.isNullAt(columnSizesPosition) ? null : wrapped.getJavaMap(columnSizesPosition); + } + + @Override + public Map valueCounts() { + return wrapped.isNullAt(valueCountsPosition) ? null : wrapped.getJavaMap(valueCountsPosition); + } + + @Override + public Map nullValueCounts() { + if (wrapped.isNullAt(nullValueCountsPosition)) { + return null; + } + return wrapped.getJavaMap(nullValueCountsPosition); + } + + @Override + public Map nanValueCounts() { + if (wrapped.isNullAt(nanValueCountsPosition)) { + return null; + } + return wrapped.getJavaMap(nanValueCountsPosition); + } + + @Override + public Map lowerBounds() { + Map lowerBounds = + wrapped.isNullAt(lowerBoundsPosition) ? null : wrapped.getJavaMap(lowerBoundsPosition); + return convert(lowerBoundsType, lowerBounds); + } + + @Override + public Map upperBounds() { + Map upperBounds = + wrapped.isNullAt(upperBoundsPosition) ? null : wrapped.getJavaMap(upperBoundsPosition); + return convert(upperBoundsType, upperBounds); + } + + @Override + public ByteBuffer keyMetadata() { + return convert(keyMetadataType, wrapped.get(keyMetadataPosition)); + } + + @Override + public F copy() { + throw new UnsupportedOperationException("Not implemented: copy"); + } + + @Override + public F copyWithoutStats() { + throw new UnsupportedOperationException("Not implemented: copyWithoutStats"); + } + + @Override + public List splitOffsets() { + return wrapped.isNullAt(splitOffsetsPosition) ? null : wrapped.getList(splitOffsetsPosition); + } + + @Override + public Integer sortOrderId() { + return wrapped.getAs(sortOrderIdPosition); + } + + @Override + public List equalityFieldIds() { + return wrapped.isNullAt(equalityIdsPosition) ? null : wrapped.getList(equalityIdsPosition); + } + + private int fieldPosition(String name, StructType sparkType) { + try { + return sparkType.fieldIndex(name); + } catch (IllegalArgumentException e) { + // the partition field is absent for unpartitioned tables + if (name.equals(DataFile.PARTITION_NAME) && wrappedPartition.size() == 0) { + return -1; + } + throw e; + } + } + + @SuppressWarnings("unchecked") + private T convert(Type valueType, Object value) { + return (T) SparkValueConverter.convert(valueType, value); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java index d8a3d5714e4b..543ebf3f9ea7 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkDataFile.java @@ -18,209 +18,29 @@ */ package org.apache.iceberg.spark; -import java.nio.ByteBuffer; import java.util.List; -import java.util.Map; import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.StructProjection; -import org.apache.spark.sql.Row; import org.apache.spark.sql.types.StructType; -public class SparkDataFile implements DataFile { - - private final int filePathPosition; - private final int fileFormatPosition; - private final int partitionPosition; - private final int recordCountPosition; - private final int fileSizeInBytesPosition; - private final int columnSizesPosition; - private final int valueCountsPosition; - private final int nullValueCountsPosition; - private final int nanValueCountsPosition; - private final int lowerBoundsPosition; - private final int upperBoundsPosition; - private final int keyMetadataPosition; - private final int splitOffsetsPosition; - private final int sortOrderIdPosition; - private final Type lowerBoundsType; - private final Type upperBoundsType; - private final Type keyMetadataType; - - private final SparkStructLike wrappedPartition; - private final StructLike partitionProjection; - private Row wrapped; +public class SparkDataFile extends SparkContentFile implements DataFile { public SparkDataFile(Types.StructType type, StructType sparkType) { - this(type, null, sparkType); + super(type, null, sparkType); } public SparkDataFile( Types.StructType type, Types.StructType projectedType, StructType sparkType) { - this.lowerBoundsType = type.fieldType(DataFile.LOWER_BOUNDS.name()); - this.upperBoundsType = type.fieldType(DataFile.UPPER_BOUNDS.name()); - this.keyMetadataType = type.fieldType(DataFile.KEY_METADATA.name()); - - Types.StructType partitionType = type.fieldType(DataFile.PARTITION_NAME).asStructType(); - this.wrappedPartition = new SparkStructLike(partitionType); - - if (projectedType != null) { - Types.StructType projectedPartitionType = - projectedType.fieldType(DataFile.PARTITION_NAME).asStructType(); - this.partitionProjection = - StructProjection.create(partitionType, projectedPartitionType).wrap(wrappedPartition); - } else { - this.partitionProjection = wrappedPartition; - } - - Map positions = Maps.newHashMap(); - type.fields() - .forEach( - field -> { - String fieldName = field.name(); - positions.put(fieldName, fieldPosition(fieldName, sparkType)); - }); - - this.filePathPosition = positions.get(DataFile.FILE_PATH.name()); - this.fileFormatPosition = positions.get(DataFile.FILE_FORMAT.name()); - this.partitionPosition = positions.get(DataFile.PARTITION_NAME); - this.recordCountPosition = positions.get(DataFile.RECORD_COUNT.name()); - this.fileSizeInBytesPosition = positions.get(DataFile.FILE_SIZE.name()); - this.columnSizesPosition = positions.get(DataFile.COLUMN_SIZES.name()); - this.valueCountsPosition = positions.get(DataFile.VALUE_COUNTS.name()); - this.nullValueCountsPosition = positions.get(DataFile.NULL_VALUE_COUNTS.name()); - this.nanValueCountsPosition = positions.get(DataFile.NAN_VALUE_COUNTS.name()); - this.lowerBoundsPosition = positions.get(DataFile.LOWER_BOUNDS.name()); - this.upperBoundsPosition = positions.get(DataFile.UPPER_BOUNDS.name()); - this.keyMetadataPosition = positions.get(DataFile.KEY_METADATA.name()); - this.splitOffsetsPosition = positions.get(DataFile.SPLIT_OFFSETS.name()); - this.sortOrderIdPosition = positions.get(DataFile.SORT_ORDER_ID.name()); + super(type, projectedType, sparkType); } - public SparkDataFile wrap(Row row) { - this.wrapped = row; - if (wrappedPartition.size() > 0) { - this.wrappedPartition.wrap(row.getAs(partitionPosition)); - } + @Override + protected DataFile asFile() { return this; } @Override - public Long pos() { + public List equalityFieldIds() { return null; } - - @Override - public int specId() { - return -1; - } - - @Override - public CharSequence path() { - return wrapped.getAs(filePathPosition); - } - - @Override - public FileFormat format() { - return FileFormat.fromString(wrapped.getString(fileFormatPosition)); - } - - @Override - public StructLike partition() { - return partitionProjection; - } - - @Override - public long recordCount() { - return wrapped.getAs(recordCountPosition); - } - - @Override - public long fileSizeInBytes() { - return wrapped.getAs(fileSizeInBytesPosition); - } - - @Override - public Map columnSizes() { - return wrapped.isNullAt(columnSizesPosition) ? null : wrapped.getJavaMap(columnSizesPosition); - } - - @Override - public Map valueCounts() { - return wrapped.isNullAt(valueCountsPosition) ? null : wrapped.getJavaMap(valueCountsPosition); - } - - @Override - public Map nullValueCounts() { - return wrapped.isNullAt(nullValueCountsPosition) - ? null - : wrapped.getJavaMap(nullValueCountsPosition); - } - - @Override - public Map nanValueCounts() { - return wrapped.isNullAt(nanValueCountsPosition) - ? null - : wrapped.getJavaMap(nanValueCountsPosition); - } - - @Override - public Map lowerBounds() { - Map lowerBounds = - wrapped.isNullAt(lowerBoundsPosition) ? null : wrapped.getJavaMap(lowerBoundsPosition); - return convert(lowerBoundsType, lowerBounds); - } - - @Override - public Map upperBounds() { - Map upperBounds = - wrapped.isNullAt(upperBoundsPosition) ? null : wrapped.getJavaMap(upperBoundsPosition); - return convert(upperBoundsType, upperBounds); - } - - @Override - public ByteBuffer keyMetadata() { - return convert(keyMetadataType, wrapped.get(keyMetadataPosition)); - } - - @Override - public DataFile copy() { - throw new UnsupportedOperationException("Not implemented: copy"); - } - - @Override - public DataFile copyWithoutStats() { - throw new UnsupportedOperationException("Not implemented: copyWithoutStats"); - } - - @Override - public List splitOffsets() { - return wrapped.isNullAt(splitOffsetsPosition) ? null : wrapped.getList(splitOffsetsPosition); - } - - @Override - public Integer sortOrderId() { - return wrapped.getAs(sortOrderIdPosition); - } - - private int fieldPosition(String name, StructType sparkType) { - try { - return sparkType.fieldIndex(name); - } catch (IllegalArgumentException e) { - // the partition field is absent for unpartitioned tables - if (name.equals(DataFile.PARTITION_NAME) && wrappedPartition.size() == 0) { - return -1; - } - throw e; - } - } - - @SuppressWarnings("unchecked") - private T convert(Type valueType, Object value) { - return (T) SparkValueConverter.convert(valueType, value); - } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java new file mode 100644 index 000000000000..6250a1630683 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkDeleteFile.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark; + +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.types.StructType; + +public class SparkDeleteFile extends SparkContentFile implements DeleteFile { + + public SparkDeleteFile(Types.StructType type, StructType sparkType) { + super(type, null, sparkType); + } + + public SparkDeleteFile( + Types.StructType type, Types.StructType projectedType, StructType sparkType) { + super(type, projectedType, sparkType); + } + + @Override + protected DeleteFile asFile() { + return this; + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 0c08324a1a84..5b1d6165695b 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -21,15 +21,19 @@ import static org.apache.iceberg.MetadataTableType.ENTRIES; import java.io.Serializable; +import java.util.Iterator; import java.util.List; import java.util.UUID; import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestWriter; @@ -50,7 +54,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.iceberg.spark.SparkContentFile; import org.apache.iceberg.spark.SparkDataFile; +import org.apache.iceberg.spark.SparkDeleteFile; import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; @@ -85,8 +91,12 @@ public class RewriteManifestsSparkAction public static final boolean USE_CACHING_DEFAULT = false; private static final Logger LOG = LoggerFactory.getLogger(RewriteManifestsSparkAction.class); + private static final RewriteManifests.Result EMPTY_RESULT = + ImmutableRewriteManifests.Result.builder() + .rewrittenManifests(ImmutableList.of()) + .addedManifests(ImmutableList.of()) + .build(); - private final Encoder manifestEncoder; private final Table table; private final int formatVersion; private final long targetManifestSizeBytes; @@ -98,7 +108,6 @@ public class RewriteManifestsSparkAction RewriteManifestsSparkAction(SparkSession spark, Table table) { super(spark); - this.manifestEncoder = Encoders.javaSerialization(ManifestFile.class); this.table = table; this.spec = table.spec(); this.targetManifestSizeBytes = @@ -159,34 +168,49 @@ public RewriteManifests.Result execute() { } private RewriteManifests.Result doExecute() { - List matchingManifests = findMatchingManifests(); + List rewrittenManifests = Lists.newArrayList(); + List addedManifests = Lists.newArrayList(); + + RewriteManifests.Result dataResult = rewriteManifests(ManifestContent.DATA); + Iterables.addAll(rewrittenManifests, dataResult.rewrittenManifests()); + Iterables.addAll(addedManifests, dataResult.addedManifests()); + + RewriteManifests.Result deletesResult = rewriteManifests(ManifestContent.DELETES); + Iterables.addAll(rewrittenManifests, deletesResult.rewrittenManifests()); + Iterables.addAll(addedManifests, deletesResult.addedManifests()); + + if (rewrittenManifests.isEmpty()) { + return EMPTY_RESULT; + } + + replaceManifests(rewrittenManifests, addedManifests); + + return ImmutableRewriteManifests.Result.builder() + .rewrittenManifests(rewrittenManifests) + .addedManifests(addedManifests) + .build(); + } + + private RewriteManifests.Result rewriteManifests(ManifestContent content) { + List matchingManifests = findMatchingManifests(content); if (matchingManifests.isEmpty()) { - return ImmutableRewriteManifests.Result.builder() - .addedManifests(ImmutableList.of()) - .rewrittenManifests(ImmutableList.of()) - .build(); + return EMPTY_RESULT; } int targetNumManifests = targetNumManifests(totalSizeBytes(matchingManifests)); - if (targetNumManifests == 1 && matchingManifests.size() == 1) { - return ImmutableRewriteManifests.Result.builder() - .addedManifests(ImmutableList.of()) - .rewrittenManifests(ImmutableList.of()) - .build(); + return EMPTY_RESULT; } Dataset manifestEntryDF = buildManifestEntryDF(matchingManifests); List newManifests; if (spec.isUnpartitioned()) { - newManifests = writeManifestsForUnpartitionedTable(manifestEntryDF, targetNumManifests); + newManifests = writeUnpartitionedManifests(content, manifestEntryDF, targetNumManifests); } else { - newManifests = writeManifestsForPartitionedTable(manifestEntryDF, targetNumManifests); + newManifests = writePartitionedManifests(content, manifestEntryDF, targetNumManifests); } - replaceManifests(matchingManifests, newManifests); - return ImmutableRewriteManifests.Result.builder() .rewrittenManifests(matchingManifests) .addedManifests(newManifests) @@ -215,41 +239,45 @@ private Dataset buildManifestEntryDF(List manifests) { .select("snapshot_id", "sequence_number", "file_sequence_number", "data_file"); } - private List writeManifestsForUnpartitionedTable( - Dataset manifestEntryDF, int numManifests) { + private List writeUnpartitionedManifests( + ManifestContent content, Dataset manifestEntryDF, int numManifests) { - StructType sparkType = (StructType) manifestEntryDF.schema().apply("data_file").dataType(); - Types.StructType combinedPartitionType = Partitioning.partitionType(table); - Types.StructType partitionType = spec.partitionType(); - - return manifestEntryDF - .repartition(numManifests) - .mapPartitions( - toManifests(manifestWriters(), combinedPartitionType, partitionType, sparkType), - manifestEncoder) - .collectAsList(); + WriteManifests writeFunc = newWriteManifestsFunc(content, manifestEntryDF.schema()); + Dataset transformedManifestEntryDF = manifestEntryDF.repartition(numManifests); + return writeFunc.apply(transformedManifestEntryDF).collectAsList(); } - private List writeManifestsForPartitionedTable( - Dataset manifestEntryDF, int numManifests) { - - StructType sparkType = (StructType) manifestEntryDF.schema().apply("data_file").dataType(); - Types.StructType combinedPartitionType = Partitioning.partitionType(table); - Types.StructType partitionType = spec.partitionType(); + private List writePartitionedManifests( + ManifestContent content, Dataset manifestEntryDF, int numManifests) { return withReusableDS( manifestEntryDF, df -> { + WriteManifests writeFunc = newWriteManifestsFunc(content, df.schema()); Column partitionColumn = df.col("data_file.partition"); - return df.repartitionByRange(numManifests, partitionColumn) - .sortWithinPartitions(partitionColumn) - .mapPartitions( - toManifests(manifestWriters(), combinedPartitionType, partitionType, sparkType), - manifestEncoder) - .collectAsList(); + Dataset transformedDF = repartitionAndSort(df, partitionColumn, numManifests); + return writeFunc.apply(transformedDF).collectAsList(); }); } + private WriteManifests newWriteManifestsFunc(ManifestContent content, StructType sparkType) { + ManifestWriterFactory writers = manifestWriters(); + + StructType sparkFileType = (StructType) sparkType.apply("data_file").dataType(); + Types.StructType combinedFileType = DataFile.getType(Partitioning.partitionType(table)); + Types.StructType fileType = DataFile.getType(spec.partitionType()); + + if (content == ManifestContent.DATA) { + return new WriteDataManifests(writers, combinedFileType, fileType, sparkFileType); + } else { + return new WriteDeleteManifests(writers, combinedFileType, fileType, sparkFileType); + } + } + + private Dataset repartitionAndSort(Dataset df, Column col, int numPartitions) { + return df.repartitionByRange(numPartitions, col).sortWithinPartitions(col); + } + private U withReusableDS(Dataset ds, Function, U> func) { boolean useCaching = PropertyUtil.propertyAsBoolean(options(), USE_CACHING, USE_CACHING_DEFAULT); @@ -264,18 +292,31 @@ private U withReusableDS(Dataset ds, Function, U> func) { } } - private List findMatchingManifests() { + private List findMatchingManifests(ManifestContent content) { Snapshot currentSnapshot = table.currentSnapshot(); if (currentSnapshot == null) { return ImmutableList.of(); } - return currentSnapshot.dataManifests(table.io()).stream() + List manifests = loadManifests(content, currentSnapshot); + + return manifests.stream() .filter(manifest -> manifest.partitionSpecId() == spec.specId() && predicate.test(manifest)) .collect(Collectors.toList()); } + private List loadManifests(ManifestContent content, Snapshot snapshot) { + switch (content) { + case DATA: + return snapshot.dataManifests(table.io()); + case DELETES: + return snapshot.deleteManifests(table.io()); + default: + throw new IllegalArgumentException("Unknown manifest content: " + content); + } + } + private int targetNumManifests(long totalSizeBytes) { return (int) ((totalSizeBytes + targetManifestSizeBytes - 1) / targetManifestSizeBytes); } @@ -339,18 +380,82 @@ private ManifestWriterFactory manifestWriters() { (long) (1.2 * targetManifestSizeBytes)); } - private static MapPartitionsFunction toManifests( - ManifestWriterFactory writers, - Types.StructType combinedPartitionType, - Types.StructType partitionType, - StructType sparkType) { + private static class WriteDataManifests extends WriteManifests { + + WriteDataManifests( + ManifestWriterFactory manifestWriters, + Types.StructType combinedPartitionType, + Types.StructType partitionType, + StructType sparkFileType) { + super(manifestWriters, combinedPartitionType, partitionType, sparkFileType); + } + + @Override + protected SparkDataFile newFileWrapper() { + return new SparkDataFile(combinedFileType(), fileType(), sparkFileType()); + } + + @Override + protected RollingManifestWriter newManifestWriter() { + return writers().newRollingManifestWriter(); + } + } + + private static class WriteDeleteManifests extends WriteManifests { + + WriteDeleteManifests( + ManifestWriterFactory manifestWriters, + Types.StructType combinedFileType, + Types.StructType fileType, + StructType sparkFileType) { + super(manifestWriters, combinedFileType, fileType, sparkFileType); + } + + @Override + protected SparkDeleteFile newFileWrapper() { + return new SparkDeleteFile(combinedFileType(), fileType(), sparkFileType()); + } + + @Override + protected RollingManifestWriter newManifestWriter() { + return writers().newRollingDeleteManifestWriter(); + } + } + + private abstract static class WriteManifests> + implements MapPartitionsFunction { + + private static final Encoder MANIFEST_ENCODER = + Encoders.javaSerialization(ManifestFile.class); + + private final ManifestWriterFactory writers; + private final Types.StructType combinedFileType; + private final Types.StructType fileType; + private final StructType sparkFileType; + + WriteManifests( + ManifestWriterFactory writers, + Types.StructType combinedFileType, + Types.StructType fileType, + StructType sparkFileType) { + this.writers = writers; + this.combinedFileType = combinedFileType; + this.fileType = fileType; + this.sparkFileType = sparkFileType; + } + + protected abstract SparkContentFile newFileWrapper(); - return rows -> { - Types.StructType combinedFileType = DataFile.getType(combinedPartitionType); - Types.StructType manifestFileType = DataFile.getType(partitionType); - SparkDataFile wrapper = new SparkDataFile(combinedFileType, manifestFileType, sparkType); + protected abstract RollingManifestWriter newManifestWriter(); - RollingManifestWriter writer = writers.newRollingManifestWriter(); + public Dataset apply(Dataset input) { + return input.mapPartitions(this, MANIFEST_ENCODER); + } + + @Override + public Iterator call(Iterator rows) throws Exception { + SparkContentFile fileWrapper = newFileWrapper(); + RollingManifestWriter writer = newManifestWriter(); try { while (rows.hasNext()) { @@ -359,14 +464,30 @@ private static MapPartitionsFunction toManifests( long sequenceNumber = row.getLong(1); Long fileSequenceNumber = row.isNullAt(2) ? null : row.getLong(2); Row file = row.getStruct(3); - writer.existing(wrapper.wrap(file), snapshotId, sequenceNumber, fileSequenceNumber); + writer.existing(fileWrapper.wrap(file), snapshotId, sequenceNumber, fileSequenceNumber); } } finally { writer.close(); } return writer.toManifestFiles().iterator(); - }; + } + + protected ManifestWriterFactory writers() { + return writers; + } + + protected Types.StructType combinedFileType() { + return combinedFileType; + } + + protected Types.StructType fileType() { + return fileType; + } + + protected StructType sparkFileType() { + return sparkFileType; + } } private static class ManifestWriterFactory implements Serializable { @@ -397,6 +518,14 @@ private ManifestWriter newManifestWriter() { return ManifestFiles.write(formatVersion, spec(), newOutputFile(), null); } + public RollingManifestWriter newRollingDeleteManifestWriter() { + return new RollingManifestWriter<>(this::newDeleteManifestWriter, maxManifestSizeBytes); + } + + private ManifestWriter newDeleteManifestWriter() { + return ManifestFiles.writeDeleteManifest(formatVersion, spec(), newOutputFile(), null); + } + private PartitionSpec spec() { return table().specs().get(specId); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index d3932c2e82a8..e7326c73e838 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -38,10 +38,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Files; +import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestWriter; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.StructLike; @@ -49,6 +54,9 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.actions.RewriteManifests; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.OutputFile; @@ -61,6 +69,8 @@ import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.iceberg.spark.source.ThreeColumnRecord; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.Pair; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; @@ -649,6 +659,255 @@ public void testRewriteLargeManifestsEvolvedUnpartitionedV1Table() throws IOExce assertThat(manifests).hasSizeGreaterThanOrEqualTo(2); } + @Test + public void testRewriteSmallDeleteManifestsNonPartitionedTable() throws IOException { + assumeThat(formatVersion).isGreaterThan(1); + + PartitionSpec spec = PartitionSpec.unpartitioned(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + // commit data records + List records = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "BBBB"), + new ThreeColumnRecord(3, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(4, "DDDDDDDDDD", "DDDD")); + writeRecords(records); + + // commit a position delete file to remove records where c1 = 1 OR c1 = 2 + List> posDeletes = generatePosDeletes("c1 = 1 OR c1 = 2"); + Pair posDeleteWriteResult = writePosDeletes(table, posDeletes); + table + .newRowDelta() + .addDeletes(posDeleteWriteResult.first()) + .validateDataFilesExist(posDeleteWriteResult.second()) + .commit(); + + // commit an equality delete file to remove all records where c1 = 3 + DeleteFile eqDeleteFile = writeEqDeletes(table, "c1", 3); + table.newRowDelta().addDeletes(eqDeleteFile).commit(); + + // the current snapshot should contain 1 data manifest and 2 delete manifests + List originalManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(originalManifests).hasSize(3); + + SparkActions actions = SparkActions.get(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + // the original delete manifests must be combined + assertThat(result.rewrittenManifests()) + .hasSize(2) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertThat(result.addedManifests()) + .hasSize(1) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertManifestsLocation(result.addedManifests()); + + // the new delete manifest must only contain files with status EXISTING + ManifestFile deleteManifest = + Iterables.getOnlyElement(table.currentSnapshot().deleteManifests(table.io())); + assertThat(deleteManifest.existingFilesCount()).isEqualTo(2); + assertThat(deleteManifest.hasAddedFiles()).isFalse(); + assertThat(deleteManifest.hasDeletedFiles()).isFalse(); + + // the preserved data manifest must only contain files with status ADDED + ManifestFile dataManifest = + Iterables.getOnlyElement(table.currentSnapshot().dataManifests(table.io())); + assertThat(dataManifest.hasExistingFiles()).isFalse(); + assertThat(dataManifest.hasAddedFiles()).isTrue(); + assertThat(dataManifest.hasDeletedFiles()).isFalse(); + + // the table must produce expected records after the rewrite + List expectedRecords = + Lists.newArrayList(new ThreeColumnRecord(4, "DDDDDDDDDD", "DDDD")); + assertThat(actualRecords()).isEqualTo(expectedRecords); + } + + @Test + public void testRewriteSmallDeleteManifestsPartitionedTable() throws IOException { + assumeThat(formatVersion).isGreaterThan(1); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + options.put(TableProperties.MANIFEST_MERGE_ENABLED, "false"); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + // commit data records + List records = + Lists.newArrayList( + new ThreeColumnRecord(1, null, "AAAA"), + new ThreeColumnRecord(2, "BBBBBBBBBB", "BBBB"), + new ThreeColumnRecord(3, "CCCCCCCCCC", "CCCC"), + new ThreeColumnRecord(4, "DDDDDDDDDD", "DDDD"), + new ThreeColumnRecord(5, "EEEEEEEEEE", "EEEE")); + writeRecords(records); + + // commit the first position delete file to remove records where c1 = 1 + List> posDeletes1 = generatePosDeletes("c1 = 1"); + Pair posDeleteWriteResult1 = + writePosDeletes(table, TestHelpers.Row.of("AAAA"), posDeletes1); + table + .newRowDelta() + .addDeletes(posDeleteWriteResult1.first()) + .validateDataFilesExist(posDeleteWriteResult1.second()) + .commit(); + + // commit the second position delete file to remove records where c1 = 2 + List> posDeletes2 = generatePosDeletes("c1 = 2"); + Pair positionDeleteWriteResult2 = + writePosDeletes(table, TestHelpers.Row.of("BBBB"), posDeletes2); + table + .newRowDelta() + .addDeletes(positionDeleteWriteResult2.first()) + .validateDataFilesExist(positionDeleteWriteResult2.second()) + .commit(); + + // commit the first equality delete file to remove records where c1 = 3 + DeleteFile eqDeleteFile1 = writeEqDeletes(table, TestHelpers.Row.of("CCCC"), "c1", 3); + table.newRowDelta().addDeletes(eqDeleteFile1).commit(); + + // commit the second equality delete file to remove records where c1 = 4 + DeleteFile eqDeleteFile2 = writeEqDeletes(table, TestHelpers.Row.of("DDDD"), "c1", 4); + table.newRowDelta().addDeletes(eqDeleteFile2).commit(); + + // the table must have 1 data manifest and 4 delete manifests + List originalManifests = table.currentSnapshot().allManifests(table.io()); + assertThat(originalManifests).hasSize(5); + + // set the target manifest size to have 2 manifests with 2 entries in each after the rewrite + List originalDeleteManifests = + table.currentSnapshot().deleteManifests(table.io()); + long manifestEntrySizeBytes = computeManifestEntrySizeBytes(originalDeleteManifests); + long targetManifestSizeBytes = (long) (1.05 * 2 * manifestEntrySizeBytes); + + table + .updateProperties() + .set(TableProperties.MANIFEST_TARGET_SIZE_BYTES, String.valueOf(targetManifestSizeBytes)) + .commit(); + + SparkActions actions = SparkActions.get(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> manifest.content() == ManifestContent.DELETES) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + // the original 4 delete manifests must be replaced with 2 new delete manifests + assertThat(result.rewrittenManifests()) + .hasSize(4) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertThat(result.addedManifests()) + .hasSize(2) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertManifestsLocation(result.addedManifests()); + + List deleteManifests = table.currentSnapshot().deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(2); + + // the first new delete manifest must only contain files with status EXISTING + ManifestFile deleteManifest1 = deleteManifests.get(0); + assertThat(deleteManifest1.existingFilesCount()).isEqualTo(2); + assertThat(deleteManifest1.hasAddedFiles()).isFalse(); + assertThat(deleteManifest1.hasDeletedFiles()).isFalse(); + + // the second new delete manifest must only contain files with status EXISTING + ManifestFile deleteManifest2 = deleteManifests.get(1); + assertThat(deleteManifest2.existingFilesCount()).isEqualTo(2); + assertThat(deleteManifest2.hasAddedFiles()).isFalse(); + assertThat(deleteManifest2.hasDeletedFiles()).isFalse(); + + // the table must produce expected records after the rewrite + List expectedRecords = + Lists.newArrayList(new ThreeColumnRecord(5, "EEEEEEEEEE", "EEEE")); + assertThat(actualRecords()).isEqualTo(expectedRecords); + } + + @Test + public void testRewriteLargeDeleteManifestsPartitionedTable() throws IOException { + assumeThat(formatVersion).isGreaterThan(1); + + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c3").build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + // generate enough delete files to have a reasonably sized manifest + List deleteFiles = Lists.newArrayList(); + for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { + DeleteFile deleteFile = newDeleteFile(table, "c3=" + fileOrdinal); + deleteFiles.add(deleteFile); + } + + // commit delete files + RowDelta rowDelta = table.newRowDelta(); + for (DeleteFile deleteFile : deleteFiles) { + rowDelta.addDeletes(deleteFile); + } + rowDelta.commit(); + + // the current snapshot should contain only 1 delete manifest + List originalDeleteManifests = + table.currentSnapshot().deleteManifests(table.io()); + ManifestFile originalDeleteManifest = Iterables.getOnlyElement(originalDeleteManifests); + + // set the target manifest size to a small value to force splitting records into multiple files + table + .updateProperties() + .set( + TableProperties.MANIFEST_TARGET_SIZE_BYTES, + String.valueOf(originalDeleteManifest.length() / 2)) + .commit(); + + SparkActions actions = SparkActions.get(); + + String stagingLocation = temp.newFolder().toString(); + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .stagingLocation(stagingLocation) + .execute(); + + // the action must rewrite the original delete manifest and add at least 2 new ones + assertThat(result.rewrittenManifests()) + .hasSize(1) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertThat(result.addedManifests()) + .hasSizeGreaterThanOrEqualTo(2) + .allMatch(m -> m.content() == ManifestContent.DELETES); + assertManifestsLocation(result.addedManifests(), stagingLocation); + + // the current snapshot must return the correct number of delete manifests + List deleteManifests = table.currentSnapshot().deleteManifests(table.io()); + assertThat(deleteManifests).hasSizeGreaterThanOrEqualTo(2); + } + + private List actualRecords() { + return spark + .read() + .format("iceberg") + .load(tableLocation) + .as(Encoders.bean(ThreeColumnRecord.class)) + .sort("c1", "c2", "c3") + .collectAsList(); + } + private void writeRecords(List records) { Dataset df = spark.createDataFrame(records, ThreeColumnRecord.class); writeDF(df); @@ -721,4 +980,63 @@ private DataFiles.Builder newDataFileBuilder(Table table) { .withFileSizeInBytes(10) .withRecordCount(1); } + + private DeleteFile newDeleteFile(Table table, String partitionPath) { + return FileMetadata.deleteFileBuilder(table.spec()) + .ofPositionDeletes() + .withPath("/path/to/pos-deletes-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(5) + .withPartitionPath(partitionPath) + .withRecordCount(1) + .build(); + } + + private List> generatePosDeletes(String predicate) { + List rows = + spark + .read() + .format("iceberg") + .load(tableLocation) + .selectExpr("_file", "_pos") + .where(predicate) + .collectAsList(); + + List> deletes = Lists.newArrayList(); + + for (Row row : rows) { + deletes.add(Pair.of(row.getString(0), row.getLong(1))); + } + + return deletes; + } + + private Pair writePosDeletes( + Table table, List> deletes) throws IOException { + return writePosDeletes(table, null, deletes); + } + + private Pair writePosDeletes( + Table table, StructLike partition, List> deletes) + throws IOException { + OutputFile outputFile = Files.localOutput(temp.newFile()); + return FileHelpers.writeDeleteFile(table, outputFile, partition, deletes); + } + + private DeleteFile writeEqDeletes(Table table, String key, Object... values) throws IOException { + return writeEqDeletes(table, null, key, values); + } + + private DeleteFile writeEqDeletes(Table table, StructLike partition, String key, Object... values) + throws IOException { + List deletes = Lists.newArrayList(); + Schema deleteSchema = table.schema().select(key); + Record delete = GenericRecord.create(deleteSchema); + + for (Object value : values) { + deletes.add(delete.copy(key, value)); + } + + OutputFile outputFile = Files.localOutput(temp.newFile()); + return FileHelpers.writeDeleteFile(table, outputFile, partition, deletes, deleteSchema); + } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java index 1f8227c94e4e..b894d32326dc 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataFile.java @@ -20,32 +20,46 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.UpdatePartitionSpec; import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.SparkDataFile; +import org.apache.iceberg.spark.SparkDeleteFile; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -57,7 +71,6 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.StructType; import org.junit.AfterClass; -import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; import org.junit.Rule; @@ -133,13 +146,13 @@ public void setupTableLocation() throws Exception { public void testValueConversion() throws IOException { Table table = TABLES.create(SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap(), tableLocation); - checkSparkDataFile(table); + checkSparkContentFiles(table); } @Test public void testValueConversionPartitionedTable() throws IOException { Table table = TABLES.create(SCHEMA, SPEC, Maps.newHashMap(), tableLocation); - checkSparkDataFile(table); + checkSparkContentFiles(table); } @Test @@ -147,10 +160,10 @@ public void testValueConversionWithEmptyStats() throws IOException { Map props = Maps.newHashMap(); props.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none"); Table table = TABLES.create(SCHEMA, SPEC, props, tableLocation); - checkSparkDataFile(table); + checkSparkContentFiles(table); } - private void checkSparkDataFile(Table table) throws IOException { + private void checkSparkContentFiles(Table table) throws IOException { Iterable rows = RandomData.generateSpark(table.schema(), 200, 0); JavaRDD rdd = sparkContext.parallelize(Lists.newArrayList(rows)); Dataset df = @@ -161,66 +174,167 @@ private void checkSparkDataFile(Table table) throws IOException { table.refresh(); + PartitionSpec dataFilesSpec = table.spec(); + List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 1 manifest", 1, manifests.size()); + assertThat(manifests).hasSize(1); List dataFiles = Lists.newArrayList(); try (ManifestReader reader = ManifestFiles.read(manifests.get(0), table.io())) { for (DataFile dataFile : reader) { - checkDataFile(dataFile.copy(), DataFiles.builder(table.spec()).copy(dataFile).build()); + checkDataFile(dataFile.copy(), DataFiles.builder(dataFilesSpec).copy(dataFile).build()); dataFiles.add(dataFile.copy()); } } - Dataset dataFileDF = spark.read().format("iceberg").load(tableLocation + "#files"); + UpdatePartitionSpec updateSpec = table.updateSpec(); + for (PartitionField field : dataFilesSpec.fields()) { + updateSpec.removeField(field.name()); + } + updateSpec.commit(); - // reorder columns to test arbitrary projections - List columns = - Arrays.stream(dataFileDF.columns()).map(ColumnName::new).collect(Collectors.toList()); - Collections.shuffle(columns); + List positionDeleteFiles = Lists.newArrayList(); + List equalityDeleteFiles = Lists.newArrayList(); + + RowDelta rowDelta = table.newRowDelta(); + + for (DataFile dataFile : dataFiles) { + DeleteFile positionDeleteFile = createPositionDeleteFile(table, dataFile); + positionDeleteFiles.add(positionDeleteFile); + rowDelta.addDeletes(positionDeleteFile); + } + + DeleteFile equalityDeleteFile1 = createEqualityDeleteFile(table); + equalityDeleteFiles.add(equalityDeleteFile1); + rowDelta.addDeletes(equalityDeleteFile1); - List sparkDataFiles = - dataFileDF.select(Iterables.toArray(columns, Column.class)).collectAsList(); + DeleteFile equalityDeleteFile2 = createEqualityDeleteFile(table); + equalityDeleteFiles.add(equalityDeleteFile2); + rowDelta.addDeletes(equalityDeleteFile2); - Assert.assertEquals( - "The number of files should match", dataFiles.size(), sparkDataFiles.size()); + rowDelta.commit(); - Types.StructType dataFileType = DataFile.getType(table.spec().partitionType()); + Dataset dataFileDF = spark.read().format("iceberg").load(tableLocation + "#data_files"); + List sparkDataFiles = shuffleColumns(dataFileDF).collectAsList(); + assertThat(sparkDataFiles).hasSameSizeAs(dataFiles); + + Types.StructType dataFileType = DataFile.getType(dataFilesSpec.partitionType()); StructType sparkDataFileType = sparkDataFiles.get(0).schema(); - SparkDataFile wrapper = new SparkDataFile(dataFileType, sparkDataFileType); + SparkDataFile dataFileWrapper = new SparkDataFile(dataFileType, sparkDataFileType); for (int i = 0; i < dataFiles.size(); i++) { - checkDataFile(dataFiles.get(i), wrapper.wrap(sparkDataFiles.get(i))); + checkDataFile(dataFiles.get(i), dataFileWrapper.wrap(sparkDataFiles.get(i))); + } + + Dataset positionDeleteFileDF = + spark.read().format("iceberg").load(tableLocation + "#delete_files").where("content = 1"); + List sparkPositionDeleteFiles = shuffleColumns(positionDeleteFileDF).collectAsList(); + assertThat(sparkPositionDeleteFiles).hasSameSizeAs(positionDeleteFiles); + + Types.StructType positionDeleteFileType = DataFile.getType(dataFilesSpec.partitionType()); + StructType sparkPositionDeleteFileType = sparkPositionDeleteFiles.get(0).schema(); + SparkDeleteFile positionDeleteFileWrapper = + new SparkDeleteFile(positionDeleteFileType, sparkPositionDeleteFileType); + + for (int i = 0; i < positionDeleteFiles.size(); i++) { + checkDeleteFile( + positionDeleteFiles.get(i), + positionDeleteFileWrapper.wrap(sparkPositionDeleteFiles.get(i))); + } + + Dataset equalityDeleteFileDF = + spark.read().format("iceberg").load(tableLocation + "#delete_files").where("content = 2"); + List sparkEqualityDeleteFiles = shuffleColumns(equalityDeleteFileDF).collectAsList(); + assertThat(sparkEqualityDeleteFiles).hasSameSizeAs(equalityDeleteFiles); + + Types.StructType equalityDeleteFileType = DataFile.getType(table.spec().partitionType()); + StructType sparkEqualityDeleteFileType = sparkEqualityDeleteFiles.get(0).schema(); + SparkDeleteFile equalityDeleteFileWrapper = + new SparkDeleteFile(equalityDeleteFileType, sparkEqualityDeleteFileType); + + for (int i = 0; i < equalityDeleteFiles.size(); i++) { + checkDeleteFile( + equalityDeleteFiles.get(i), + equalityDeleteFileWrapper.wrap(sparkEqualityDeleteFiles.get(i))); } } + private Dataset shuffleColumns(Dataset df) { + List columns = + Arrays.stream(df.columns()).map(ColumnName::new).collect(Collectors.toList()); + Collections.shuffle(columns); + return df.select(columns.toArray(new Column[0])); + } + private void checkDataFile(DataFile expected, DataFile actual) { - Assert.assertEquals("Path must match", expected.path(), actual.path()); - Assert.assertEquals("Format must match", expected.format(), actual.format()); - Assert.assertEquals("Record count must match", expected.recordCount(), actual.recordCount()); - Assert.assertEquals("Size must match", expected.fileSizeInBytes(), actual.fileSizeInBytes()); - Assert.assertEquals( - "Record value counts must match", expected.valueCounts(), actual.valueCounts()); - Assert.assertEquals( - "Record null value counts must match", - expected.nullValueCounts(), - actual.nullValueCounts()); - Assert.assertEquals( - "Record nan value counts must match", expected.nanValueCounts(), actual.nanValueCounts()); - Assert.assertEquals("Lower bounds must match", expected.lowerBounds(), actual.lowerBounds()); - Assert.assertEquals("Upper bounds must match", expected.upperBounds(), actual.upperBounds()); - Assert.assertEquals("Key metadata must match", expected.keyMetadata(), actual.keyMetadata()); - Assert.assertEquals("Split offsets must match", expected.splitOffsets(), actual.splitOffsets()); - Assert.assertEquals("Sort order id must match", expected.sortOrderId(), actual.sortOrderId()); + assertThat(expected.equalityFieldIds()).isNull(); + assertThat(actual.equalityFieldIds()).isNull(); + checkContentFile(expected, actual); + checkStructLike(expected.partition(), actual.partition()); + } + private void checkDeleteFile(DeleteFile expected, DeleteFile actual) { + assertThat(expected.equalityFieldIds()).isEqualTo(actual.equalityFieldIds()); + checkContentFile(expected, actual); checkStructLike(expected.partition(), actual.partition()); } + private void checkContentFile(ContentFile expected, ContentFile actual) { + assertThat(actual.content()).isEqualTo(expected.content()); + assertThat(actual.path()).isEqualTo(expected.path()); + assertThat(actual.format()).isEqualTo(expected.format()); + assertThat(actual.recordCount()).isEqualTo(expected.recordCount()); + assertThat(actual.fileSizeInBytes()).isEqualTo(expected.fileSizeInBytes()); + assertThat(actual.valueCounts()).isEqualTo(expected.valueCounts()); + assertThat(actual.nullValueCounts()).isEqualTo(expected.nullValueCounts()); + assertThat(actual.nanValueCounts()).isEqualTo(expected.nanValueCounts()); + assertThat(actual.lowerBounds()).isEqualTo(expected.lowerBounds()); + assertThat(actual.upperBounds()).isEqualTo(expected.upperBounds()); + assertThat(actual.keyMetadata()).isEqualTo(expected.keyMetadata()); + assertThat(actual.splitOffsets()).isEqualTo(expected.splitOffsets()); + assertThat(actual.sortOrderId()).isEqualTo(expected.sortOrderId()); + } + private void checkStructLike(StructLike expected, StructLike actual) { - Assert.assertEquals("Struct size should match", expected.size(), actual.size()); + assertThat(actual.size()).isEqualTo(expected.size()); for (int i = 0; i < expected.size(); i++) { - Assert.assertEquals( - "Struct values must match", expected.get(i, Object.class), actual.get(i, Object.class)); + assertThat(actual.get(i, Object.class)).isEqualTo(expected.get(i, Object.class)); } } + + private DeleteFile createPositionDeleteFile(Table table, DataFile dataFile) { + PartitionSpec spec = table.specs().get(dataFile.specId()); + return FileMetadata.deleteFileBuilder(spec) + .ofPositionDeletes() + .withPath("/path/to/pos-deletes-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(dataFile.fileSizeInBytes() / 4) + .withPartition(dataFile.partition()) + .withRecordCount(2) + .withMetrics( + new Metrics( + 2L, + null, // no column sizes + null, // no value counts + null, // no null counts + null, // no NaN counts + ImmutableMap.of( + MetadataColumns.DELETE_FILE_PATH.fieldId(), + Conversions.toByteBuffer(Types.StringType.get(), dataFile.path())), + ImmutableMap.of( + MetadataColumns.DELETE_FILE_PATH.fieldId(), + Conversions.toByteBuffer(Types.StringType.get(), dataFile.path())))) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .build(); + } + + private DeleteFile createEqualityDeleteFile(Table table) { + return FileMetadata.deleteFileBuilder(table.spec()) + .ofEqualityDeletes(3, 4) + .withPath("/path/to/eq-deletes-" + UUID.randomUUID() + ".parquet") + .withFileSizeInBytes(250) + .withRecordCount(1) + .withSortOrder(SortOrder.unsorted()) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .build(); + } } From b6b5d4473f7ecbd0961caf528728ef024f70a927 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 19 Nov 2023 07:12:36 +0100 Subject: [PATCH 44/94] Build: Bump datamodel-code-generator from 0.23.0 to 0.24.2 (#9109) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.23.0 to 0.24.2. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.23.0...0.24.2) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index 6fa2df8d82c5..d1e87bd8011f 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,6 +16,6 @@ # under the License. openapi-spec-validator==0.5.2 -datamodel-code-generator==0.23.0 +datamodel-code-generator==0.24.2 # Add the Pydantic constraint since 2.4.0 has a bug pydantic<2.4.0 From 8c7ebeee82b59904753fb6eab61a1e28e9852ae3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 19 Nov 2023 07:57:53 +0100 Subject: [PATCH 45/94] Build: Bump openapi-spec-validator from 0.5.2 to 0.7.1 (#9057) --- open-api/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index d1e87bd8011f..1208f5ca4482 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -15,7 +15,7 @@ # specific language governing permissions and limitations # under the License. -openapi-spec-validator==0.5.2 +openapi-spec-validator==0.7.1 datamodel-code-generator==0.24.2 # Add the Pydantic constraint since 2.4.0 has a bug pydantic<2.4.0 From 9e94dc85764dd1321c9782a2a0d834a9bfd84b7d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sun, 19 Nov 2023 10:39:06 +0100 Subject: [PATCH 46/94] Open-API: Remove pydantic pin (#9110) --- open-api/requirements.txt | 2 -- 1 file changed, 2 deletions(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index 1208f5ca4482..a9be79a486fb 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -17,5 +17,3 @@ openapi-spec-validator==0.7.1 datamodel-code-generator==0.24.2 -# Add the Pydantic constraint since 2.4.0 has a bug -pydantic<2.4.0 From 573f3f30da1d9739547f8f98c4a0d61613d26412 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 19 Nov 2023 10:39:44 +0100 Subject: [PATCH 47/94] Build: Bump com.fasterxml.jackson.core:jackson-annotations (#9106) Bumps [com.fasterxml.jackson.core:jackson-annotations](https://github.com/FasterXML/jackson) from 2.15.3 to 2.16.0. - [Commits](https://github.com/FasterXML/jackson/commits) --- updated-dependencies: - dependency-name: com.fasterxml.jackson.core:jackson-annotations dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index aff898260ac0..400d00ae88b6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -47,7 +47,7 @@ httpcomponents-httpclient5 = "5.2.1" hive2 = { strictly = "[2, 3[", prefer = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" immutables-value = "2.10.0" -jackson-annotations = "2.15.3" +jackson-annotations = "2.16.0" jackson-bom = "2.14.2" jackson-dataformat-xml = "2.9.9" jackson211 = { strictly = "[2.11, 2.12[", prefer = "2.11.4"} # see rich version usage explanation above From a3bf0c7e328a8d514c9918cc80cc775169d6b53b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 19 Nov 2023 10:40:02 +0100 Subject: [PATCH 48/94] Build: Bump org.testcontainers:testcontainers from 1.19.1 to 1.19.2 (#9103) Bumps [org.testcontainers:testcontainers](https://github.com/testcontainers/testcontainers-java) from 1.19.1 to 1.19.2. - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/1.19.1...1.19.2) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 400d00ae88b6..471440356817 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -83,7 +83,7 @@ spark-hive35 = "3.5.0" spring-boot = "2.5.4" spring-web = "5.3.30" sqlite-jdbc = "3.44.0.0" -testcontainers = "1.19.1" +testcontainers = "1.19.2" tez010 = "0.10.2" tez08 = { strictly = "[0.8, 0.9[", prefer = "0.8.4"} # see rich version usage explanation above From 04d1a9d85e2732d885c435085e53b7fc7318379a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 19 Nov 2023 10:40:46 +0100 Subject: [PATCH 49/94] Build: Bump com.fasterxml.jackson.dataformat:jackson-dataformat-xml (#9107) Bumps [com.fasterxml.jackson.dataformat:jackson-dataformat-xml](https://github.com/FasterXML/jackson-dataformat-xml) from 2.9.9 to 2.16.0. - [Commits](https://github.com/FasterXML/jackson-dataformat-xml/compare/jackson-dataformat-xml-2.9.9...jackson-dataformat-xml-2.16.0) --- updated-dependencies: - dependency-name: com.fasterxml.jackson.dataformat:jackson-dataformat-xml dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 471440356817..aa7aa54f5d7b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -49,7 +49,7 @@ hive3 = "3.1.3" immutables-value = "2.10.0" jackson-annotations = "2.16.0" jackson-bom = "2.14.2" -jackson-dataformat-xml = "2.9.9" +jackson-dataformat-xml = "2.16.0" jackson211 = { strictly = "[2.11, 2.12[", prefer = "2.11.4"} # see rich version usage explanation above jackson212 = { strictly = "[2.12, 2.13[", prefer = "2.12.3"} jackson213 = { strictly = "[2.13, 2.14[", prefer = "2.13.4"} From c193de9e8f6ef8195506eeebe4a3721cf9884bd9 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Sun, 19 Nov 2023 15:56:55 -0800 Subject: [PATCH 50/94] Spark: Fix metadata delete check with branches (#9102) --- .../spark/extensions/SparkRowLevelOperationsTestBase.java | 2 +- .../java/org/apache/iceberg/spark/extensions/TestDelete.java | 3 +-- .../main/java/org/apache/iceberg/spark/source/SparkTable.java | 2 +- .../test/java/org/apache/iceberg/spark/data/TestHelpers.java | 2 +- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 16ab98f4424c..e4605a765823 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -135,7 +135,7 @@ public static Object[][] parameters() { true, WRITE_DISTRIBUTION_MODE_NONE, false, - null, + "test", DISTRIBUTED }, { diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index bdc4be08b4ee..9bc46d05713f 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -326,9 +326,8 @@ public void testDeleteWithoutScanningTable() throws Exception { public void testDeleteFileThenMetadataDelete() throws Exception { Assume.assumeFalse("Avro does not support metadata delete", fileFormat.equals("avro")); createAndInitUnpartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); createBranchIfNeeded(); + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", commitTarget()); // MOR mode: writes a delete file as null cannot be deleted by metadata sql("DELETE FROM %s AS t WHERE t.id IS NULL", commitTarget()); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index b540165b2ec3..e200bee03e9f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -330,7 +330,7 @@ private boolean canDeleteUsingMetadata(Expression deleteExpr) { .ignoreResiduals(); if (branch != null) { - scan.useRef(branch); + scan = scan.useRef(branch); } try (CloseableIterable tasks = scan.planFiles()) { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index ca9d14f05e59..8e6b576ddffb 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -806,7 +806,7 @@ public static List dataFiles(Table table) { public static List dataFiles(Table table, String branch) { TableScan scan = table.newScan(); if (branch != null) { - scan.useRef(branch); + scan = scan.useRef(branch); } CloseableIterable tasks = scan.includeColumnStats().planFiles(); From f8d21116b1990ff0f7d7960a0f41f3a807756141 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 20 Nov 2023 08:38:33 +0100 Subject: [PATCH 51/94] Build: Bump software.amazon.awssdk:bom from 2.21.22 to 2.21.26 (#9105) Bumps software.amazon.awssdk:bom from 2.21.22 to 2.21.26. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index aa7aa54f5d7b..3b27bad336c8 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -27,7 +27,7 @@ arrow = "14.0.1" avro = "1.11.3" assertj-core = "3.24.2" awaitility = "4.2.0" -awssdk-bom = "2.21.22" +awssdk-bom = "2.21.26" azuresdk-bom = "1.2.18" caffeine = "2.9.3" calcite = "1.10.0" From 3f90a23c1a545a5cf091f74c3f6e96f53fe41869 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 20 Nov 2023 21:42:08 +0100 Subject: [PATCH 52/94] Build: Bump mkdocs-material from 9.4.8 to 9.4.10 (#9114) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.4.8 to 9.4.10. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.4.8...9.4.10) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index a3fc59f97d44..6b5850500871 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.4.8 +mkdocs-material==9.4.10 mkdocs-material-extensions==1.3 mkdocs-monorepo-plugin==1.0.5 mkdocs-redirects==1.2.1 From 7320899deab780597c101beddc8561adde7938f4 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 20 Nov 2023 14:06:54 -0800 Subject: [PATCH 53/94] Spark 3.3, 3.4: Backport fix for metadata delete condition check for branches (#9115) --- .../spark/extensions/SparkRowLevelOperationsTestBase.java | 2 +- .../java/org/apache/iceberg/spark/extensions/TestDelete.java | 3 +-- .../main/java/org/apache/iceberg/spark/source/SparkTable.java | 2 +- .../test/java/org/apache/iceberg/spark/data/TestHelpers.java | 2 +- .../spark/extensions/SparkRowLevelOperationsTestBase.java | 2 +- .../java/org/apache/iceberg/spark/extensions/TestDelete.java | 3 +-- .../main/java/org/apache/iceberg/spark/source/SparkTable.java | 2 +- .../test/java/org/apache/iceberg/spark/data/TestHelpers.java | 2 +- 8 files changed, 8 insertions(+), 10 deletions(-) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 0ca63ae2bfa2..ceb6b6aad87e 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -114,7 +114,7 @@ public static Object[][] parameters() { "parquet", true, WRITE_DISTRIBUTION_MODE_NONE, - null, + "test", }, { "testhadoop", diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index c5ff25b1a094..6020e00b3235 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -144,9 +144,8 @@ public void testDeleteWithoutScanningTable() throws Exception { public void testDeleteFileThenMetadataDelete() throws Exception { Assume.assumeFalse("Avro does not support metadata delete", fileFormat.equals("avro")); createAndInitUnpartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); createBranchIfNeeded(); + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", commitTarget()); // MOR mode: writes a delete file as null cannot be deleted by metadata sql("DELETE FROM %s AS t WHERE t.id IS NULL", commitTarget()); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 74c3667f6f65..574d014e8335 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -330,7 +330,7 @@ private boolean canDeleteUsingMetadata(Expression deleteExpr) { .ignoreResiduals(); if (branch != null) { - scan.useRef(branch); + scan = scan.useRef(branch); } try (CloseableIterable tasks = scan.planFiles()) { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 82d0f0daf595..f2633fc67640 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -790,7 +790,7 @@ public static List dataFiles(Table table) { public static List dataFiles(Table table, String branch) { TableScan scan = table.newScan(); if (branch != null) { - scan.useRef(branch); + scan = scan.useRef(branch); } CloseableIterable tasks = scan.includeColumnStats().planFiles(); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 16ab98f4424c..e4605a765823 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -135,7 +135,7 @@ public static Object[][] parameters() { true, WRITE_DISTRIBUTION_MODE_NONE, false, - null, + "test", DISTRIBUTED }, { diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index 87a98cbae5f8..b901d567f646 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -312,9 +312,8 @@ public void testDeleteWithoutScanningTable() throws Exception { public void testDeleteFileThenMetadataDelete() throws Exception { Assume.assumeFalse("Avro does not support metadata delete", fileFormat.equals("avro")); createAndInitUnpartitionedTable(); - - sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", tableName); createBranchIfNeeded(); + sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", commitTarget()); // MOR mode: writes a delete file as null cannot be deleted by metadata sql("DELETE FROM %s AS t WHERE t.id IS NULL", commitTarget()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index b540165b2ec3..e200bee03e9f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -330,7 +330,7 @@ private boolean canDeleteUsingMetadata(Expression deleteExpr) { .ignoreResiduals(); if (branch != null) { - scan.useRef(branch); + scan = scan.useRef(branch); } try (CloseableIterable tasks = scan.planFiles()) { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index ca9d14f05e59..8e6b576ddffb 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -806,7 +806,7 @@ public static List dataFiles(Table table) { public static List dataFiles(Table table, String branch) { TableScan scan = table.newScan(); if (branch != null) { - scan.useRef(branch); + scan = scan.useRef(branch); } CloseableIterable tasks = scan.includeColumnStats().planFiles(); From 506cdbfd5309963d84094b58cd58a4d6c97a3cc5 Mon Sep 17 00:00:00 2001 From: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com> Date: Tue, 21 Nov 2023 07:08:59 +0800 Subject: [PATCH 54/94] Spark: Add SQL config to control locality (#9101) --- .../main/java/org/apache/iceberg/spark/SparkReadConf.java | 7 ++++++- .../java/org/apache/iceberg/spark/SparkSQLProperties.java | 3 +++ .../main/java/org/apache/iceberg/spark/SparkReadConf.java | 7 ++++++- .../java/org/apache/iceberg/spark/SparkSQLProperties.java | 3 +++ .../main/java/org/apache/iceberg/spark/SparkReadConf.java | 7 ++++++- .../java/org/apache/iceberg/spark/SparkSQLProperties.java | 3 +++ 6 files changed, 27 insertions(+), 3 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index 0f24844414fe..0d43fb957306 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -71,7 +71,12 @@ public boolean caseSensitive() { public boolean localityEnabled() { boolean defaultValue = Util.mayHaveBlockLocations(table.io(), table.location()); - return PropertyUtil.propertyAsBoolean(readOptions, SparkReadOptions.LOCALITY, defaultValue); + return confParser + .booleanConf() + .option(SparkReadOptions.LOCALITY) + .sessionConf(SparkSQLProperties.LOCALITY) + .defaultValue(defaultValue) + .parse(); } public Long snapshotId() { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index d7ff4311c907..1d1fa885bbe3 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -63,4 +63,7 @@ private SparkSQLProperties() {} // Controls the WAP branch used for write-audit-publish workflow. // When set, new snapshots will be committed to this branch. public static final String WAP_BRANCH = "spark.wap.branch"; + + // Controls whether to report locality information to Spark while allocating input partitions + public static final String LOCALITY = "spark.sql.iceberg.locality.enabled"; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index be0ba7d6bc46..f05a4530859c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -81,7 +81,12 @@ public boolean caseSensitive() { public boolean localityEnabled() { boolean defaultValue = Util.mayHaveBlockLocations(table.io(), table.location()); - return PropertyUtil.propertyAsBoolean(readOptions, SparkReadOptions.LOCALITY, defaultValue); + return confParser + .booleanConf() + .option(SparkReadOptions.LOCALITY) + .sessionConf(SparkSQLProperties.LOCALITY) + .defaultValue(defaultValue) + .parse(); } public Long snapshotId() { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index 9accd6f108dc..217665019115 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -74,4 +74,7 @@ private SparkSQLProperties() {} // Overrides the delete planning mode public static final String DELETE_PLANNING_MODE = "spark.sql.iceberg.delete-planning-mode"; + + // Controls whether to report locality information to Spark while allocating input partitions + public static final String LOCALITY = "spark.sql.iceberg.locality.enabled"; } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index 6b657587f930..984e2bce1efc 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -79,7 +79,12 @@ public boolean caseSensitive() { public boolean localityEnabled() { boolean defaultValue = Util.mayHaveBlockLocations(table.io(), table.location()); - return PropertyUtil.propertyAsBoolean(readOptions, SparkReadOptions.LOCALITY, defaultValue); + return confParser + .booleanConf() + .option(SparkReadOptions.LOCALITY) + .sessionConf(SparkSQLProperties.LOCALITY) + .defaultValue(defaultValue) + .parse(); } public Long snapshotId() { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index 11d9cc296cbe..bca41b4155ed 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -67,4 +67,7 @@ private SparkSQLProperties() {} // Overrides the advisory partition size public static final String ADVISORY_PARTITION_SIZE = "spark.sql.iceberg.advisory-partition-size"; + + // Controls whether to report locality information to Spark while allocating input partitions + public static final String LOCALITY = "spark.sql.iceberg.locality.enabled"; } From 42614cc8d6527284cb45800b5b37e6a98c466ef6 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 20 Nov 2023 23:25:56 -0800 Subject: [PATCH 55/94] Core: Remove synchronization from BitmapPositionDeleteIndex (#9119) --- .../org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java index 1aa6571a03f9..7690ab7e4879 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java @@ -28,7 +28,7 @@ class BitmapPositionDeleteIndex implements PositionDeleteIndex { } @Override - public synchronized void delete(long position) { + public void delete(long position) { roaring64Bitmap.add(position); } From c61c3ca017aaec145f57948f0da9cb307bf45cde Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 21 Nov 2023 12:20:41 -0800 Subject: [PATCH 56/94] Data: Always use delete index for position deletes (#9117) --- .../org/apache/iceberg/data/DeleteFilter.java | 18 +++--------------- 1 file changed, 3 insertions(+), 15 deletions(-) diff --git a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java index a7979fd2ed3e..55acd3200894 100644 --- a/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java +++ b/data/src/main/java/org/apache/iceberg/data/DeleteFilter.java @@ -58,11 +58,9 @@ public abstract class DeleteFilter { private static final Logger LOG = LoggerFactory.getLogger(DeleteFilter.class); - private static final long DEFAULT_SET_FILTER_THRESHOLD = 100_000L; private static final Schema POS_DELETE_SCHEMA = new Schema(MetadataColumns.DELETE_FILE_PATH, MetadataColumns.DELETE_FILE_POS); - private final long setFilterThreshold; private final String filePath; private final List posDeletes; private final List eqDeletes; @@ -82,7 +80,6 @@ protected DeleteFilter( Schema tableSchema, Schema requestedSchema, DeleteCounter counter) { - this.setFilterThreshold = DEFAULT_SET_FILTER_THRESHOLD; this.filePath = filePath; this.counter = counter; @@ -245,18 +242,9 @@ private CloseableIterable applyPosDeletes(CloseableIterable records) { List> deletes = Lists.transform(posDeletes, this::openPosDeletes); - // if there are fewer deletes than a reasonable number to keep in memory, use a set - if (posDeletes.stream().mapToLong(DeleteFile::recordCount).sum() < setFilterThreshold) { - PositionDeleteIndex positionIndex = Deletes.toPositionIndex(filePath, deletes); - Predicate isDeleted = record -> positionIndex.isDeleted(pos(record)); - return createDeleteIterable(records, isDeleted); - } - - return hasIsDeletedColumn - ? Deletes.streamingMarker( - records, this::pos, Deletes.deletePositions(filePath, deletes), this::markRowDeleted) - : Deletes.streamingFilter( - records, this::pos, Deletes.deletePositions(filePath, deletes), counter); + PositionDeleteIndex positionIndex = Deletes.toPositionIndex(filePath, deletes); + Predicate isDeleted = record -> positionIndex.isDeleted(pos(record)); + return createDeleteIterable(records, isDeleted); } private CloseableIterable createDeleteIterable( From 229a243ddad20a725d9f7e4446ce8bb3fc2c9a53 Mon Sep 17 00:00:00 2001 From: przemekd <1896041+przemekd@users.noreply.github.com> Date: Wed, 22 Nov 2023 05:53:23 +0100 Subject: [PATCH 57/94] Core: Lazily create LocationProvider in SerializableTable (#9029) --- .../org/apache/iceberg/SerializableTable.java | 22 ++++++++++++------- 1 file changed, 14 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index c5085b1313c9..778a993c5144 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -41,10 +41,10 @@ * table metadata, it directly persists the current schema, spec, sort order, table properties to * avoid reading the metadata file from other nodes for frequently needed metadata. * - *

The implementation assumes the passed instances of {@link FileIO}, {@link EncryptionManager}, - * {@link LocationProvider} are serializable. If you are serializing the table using a custom - * serialization framework like Kryo, those instances of {@link FileIO}, {@link EncryptionManager}, - * {@link LocationProvider} must be supported by that particular serialization framework. + *

The implementation assumes the passed instances of {@link FileIO}, {@link EncryptionManager} + * are serializable. If you are serializing the table using a custom serialization framework like + * Kryo, those instances of {@link FileIO}, {@link EncryptionManager} must be supported by that + * particular serialization framework. * *

Note: loading the complete metadata from a large number of nodes can overwhelm the * storage. @@ -61,9 +61,9 @@ public class SerializableTable implements Table, Serializable { private final String sortOrderAsJson; private final FileIO io; private final EncryptionManager encryption; - private final LocationProvider locationProvider; private final Map refs; + private transient volatile LocationProvider lazyLocationProvider = null; private transient volatile Table lazyTable = null; private transient volatile Schema lazySchema = null; private transient volatile Map lazySpecs = null; @@ -83,7 +83,6 @@ protected SerializableTable(Table table) { this.sortOrderAsJson = SortOrderParser.toJson(table.sortOrder()); this.io = fileIO(table); this.encryption = table.encryption(); - this.locationProvider = table.locationProvider(); this.refs = SerializableMap.copyOf(table.refs()); this.uuid = table.uuid(); } @@ -129,7 +128,7 @@ private Table lazyTable() { } TableOperations ops = - new StaticTableOperations(metadataFileLocation, io, locationProvider); + new StaticTableOperations(metadataFileLocation, io, locationProvider()); this.lazyTable = newTable(ops, name); } } @@ -237,7 +236,14 @@ public EncryptionManager encryption() { @Override public LocationProvider locationProvider() { - return locationProvider; + if (lazyLocationProvider == null) { + synchronized (this) { + if (lazyLocationProvider == null) { + this.lazyLocationProvider = LocationProviders.locationsFor(location, properties); + } + } + } + return lazyLocationProvider; } @Override From 0831eb03b09cce2d09dd62da829323004b41f423 Mon Sep 17 00:00:00 2001 From: pvary Date: Thu, 23 Nov 2023 10:40:33 +0100 Subject: [PATCH 58/94] Flink: Emit watermarks from the IcebergSource (#8553) --- .../iceberg/data/GenericAppenderHelper.java | 5 + .../iceberg/flink/source/IcebergSource.java | 68 ++- .../reader/ColumnStatsWatermarkExtractor.java | 98 ++++ .../source/reader/IcebergSourceReader.java | 3 +- .../reader/SerializableRecordEmitter.java | 40 ++ ...tter.java => SplitWatermarkExtractor.java} | 18 +- .../WatermarkExtractorRecordEmitter.java | 67 +++ .../flink/source/split/SplitComparators.java | 19 +- .../source/TestIcebergSourceFailover.java | 12 +- ...gSourceFailoverWithWatermarkExtractor.java | 112 +++++ ...stIcebergSourceWithWatermarkExtractor.java | 451 ++++++++++++++++++ .../assigner/SplitAssignerTestBase.java | 21 +- ...tFileSequenceNumberBasedSplitAssigner.java | 10 +- .../TestWatermarkBasedSplitAssigner.java | 146 ++++++ .../flink/source/reader/ReaderUtil.java | 8 +- .../TestColumnStatsWatermarkExtractor.java | 178 +++++++ .../reader/TestIcebergSourceReader.java | 7 +- 17 files changed, 1220 insertions(+), 43 deletions(-) create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/{IcebergSourceRecordEmitter.java => SplitWatermarkExtractor.java} (63%) create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java diff --git a/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java b/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java index 03e220f10ed3..beccc8bb70bf 100644 --- a/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java +++ b/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java @@ -39,6 +39,7 @@ public class GenericAppenderHelper { private static final String ORC_CONFIG_PREFIX = "^orc.*"; + private static final String PARQUET_CONFIG_PATTERN = ".*parquet.*"; private final Table table; private final FileFormat fileFormat; @@ -120,6 +121,10 @@ private static DataFile appendToLocalFile( appenderFactory.setAll(conf.getValByRegex(ORC_CONFIG_PREFIX)); } + if (FileFormat.PARQUET.equals(format) && conf != null) { + appenderFactory.setAll(conf.getValByRegex(PARQUET_CONFIG_PATTERN)); + } + FileAppender appender = appenderFactory.newAppender(Files.localOutput(file), format); try (FileAppender fileAppender = appender) { fileAppender.addAll(records); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index f85f6277263b..179253cb3a18 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.flink.annotation.Experimental; import org.apache.flink.api.connector.source.Boundedness; @@ -58,15 +59,20 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; +import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,6 +86,7 @@ public class IcebergSource implements Source readerFunction; private final SplitAssignerFactory assignerFactory; private final SerializableComparator splitComparator; + private final SerializableRecordEmitter emitter; // Can't use SerializableTable as enumerator needs a regular table // that can discover table changes @@ -91,13 +98,15 @@ public class IcebergSource implements Source readerFunction, SplitAssignerFactory assignerFactory, SerializableComparator splitComparator, - Table table) { + Table table, + SerializableRecordEmitter emitter) { this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; this.splitComparator = splitComparator; this.table = table; + this.emitter = emitter; } String name() { @@ -152,7 +161,8 @@ public Boundedness getBoundedness() { public SourceReader createReader(SourceReaderContext readerContext) { IcebergSourceReaderMetrics metrics = new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); - return new IcebergSourceReader<>(metrics, readerFunction, splitComparator, readerContext); + return new IcebergSourceReader<>( + emitter, metrics, readerFunction, splitComparator, readerContext); } @Override @@ -216,6 +226,8 @@ public static class Builder { private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; + private String watermarkColumn; + private TimeUnit watermarkTimeUnit = TimeUnit.MICROSECONDS; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -237,6 +249,9 @@ public Builder table(Table newTable) { } public Builder assignerFactory(SplitAssignerFactory assignerFactory) { + Preconditions.checkArgument( + watermarkColumn == null, + "Watermark column and SplitAssigner should not be set in the same source"); this.splitAssignerFactory = assignerFactory; return this; } @@ -429,6 +444,33 @@ public Builder setAll(Map properties) { return this; } + /** + * Emits watermarks once per split based on the min value of column statistics from files + * metadata in the given split. The generated watermarks are also used for ordering the splits + * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider + * setting {@link #watermarkTimeUnit(TimeUnit)}. + * + *

Consider setting `read.split.open-file-cost` to prevent combining small files to a single + * split when the watermark is used for watermark alignment. + */ + public Builder watermarkColumn(String columnName) { + Preconditions.checkArgument( + splitAssignerFactory == null, + "Watermark column and SplitAssigner should not be set in the same source"); + this.watermarkColumn = columnName; + return this; + } + + /** + * When the type of the {@link #watermarkColumn} is {@link + * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the + * value. The default value is {@link TimeUnit#MICROSECONDS}. + */ + public Builder watermarkTimeUnit(TimeUnit timeUnit) { + this.watermarkTimeUnit = timeUnit; + return this; + } + /** @deprecated Use {@link #setAll} instead. */ @Deprecated public Builder properties(Map properties) { @@ -453,6 +495,18 @@ public IcebergSource build() { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); } + SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); + if (watermarkColumn != null) { + // Column statistics is needed for watermark generation + contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); + + SplitWatermarkExtractor watermarkExtractor = + new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); + emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); + splitAssignerFactory = + new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); + } + ScanContext context = contextBuilder.build(); if (readerFunction == null) { if (table instanceof BaseMetadataTable) { @@ -485,8 +539,14 @@ public IcebergSource build() { checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading - return new IcebergSource( - tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table); + return new IcebergSource<>( + tableLoader, + context, + readerFunction, + splitAssignerFactory, + splitComparator, + table, + emitter); } private void checkRequired() { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..4bb6f0a98c4c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; + +/** + * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics + * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link + * WatermarkExtractorRecordEmitter} along with the actual records. + */ +@Internal +public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { + private final int eventTimeFieldId; + private final String eventTimeFieldName; + private final TimeUnit timeUnit; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param eventTimeFieldName The column which should be used as an event time + * @param timeUnit Used for converting the long value to epoch milliseconds + */ + public ColumnStatsWatermarkExtractor( + Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { + Types.NestedField field = schema.findField(eventTimeFieldName); + TypeID typeID = field.type().typeId(); + Preconditions.checkArgument( + typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), + "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", + typeID); + this.eventTimeFieldId = field.fieldId(); + this.eventTimeFieldName = eventTimeFieldName; + // Use the timeUnit only for Long columns. + this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; + } + + @VisibleForTesting + ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { + this.eventTimeFieldId = eventTimeFieldId; + this.eventTimeFieldName = eventTimeFieldName; + this.timeUnit = TimeUnit.MICROSECONDS; + } + + /** + * Get the watermark for a split using column statistics. + * + * @param split The split + * @return The watermark + * @throws IllegalArgumentException if there is no statistics for the column + */ + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> { + Preconditions.checkArgument( + scanTask.file().lowerBounds() != null + && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, + "Missing statistics for column name = %s in file = %s", + eventTimeFieldName, + eventTimeFieldId, + scanTask.file()); + return timeUnit.toMillis( + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); + }) + .min(Comparator.comparingLong(l -> l)) + .get(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java index 8d7d68f961cb..f143b8d2df2e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java @@ -35,13 +35,14 @@ public class IcebergSourceReader RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { public IcebergSourceReader( + SerializableRecordEmitter emitter, IcebergSourceReaderMetrics metrics, ReaderFunction readerFunction, SerializableComparator splitComparator, SourceReaderContext context) { super( () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), - new IcebergSourceRecordEmitter<>(), + emitter, context.getConfiguration(), context); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java new file mode 100644 index 000000000000..a6e2c1dae243 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +@Internal +@FunctionalInterface +public interface SerializableRecordEmitter + extends RecordEmitter, T, IcebergSourceSplit>, Serializable { + static SerializableRecordEmitter defaultEmitter() { + return (element, output, split) -> { + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + }; + } + + static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { + return new WatermarkExtractorRecordEmitter<>(extractor); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 63% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java index 337d9d3c4223..d1c50ac8ca52 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java @@ -18,19 +18,11 @@ */ package org.apache.iceberg.flink.source.reader; -import org.apache.flink.api.connector.source.SourceOutput; -import org.apache.flink.connector.base.source.reader.RecordEmitter; +import java.io.Serializable; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -final class IcebergSourceRecordEmitter - implements RecordEmitter, T, IcebergSourceSplit> { - - IcebergSourceRecordEmitter() {} - - @Override - public void emitRecord( - RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - } +/** The interface used to extract watermarks from splits. */ +public interface SplitWatermarkExtractor extends Serializable { + /** Get the watermark for a split. */ + long extractWatermark(IcebergSourceSplit split); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java new file mode 100644 index 000000000000..02ef57d344b1 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the watermarks, records and updates the split position. + * + *

The Emitter emits watermarks at the beginning of every split provided by the {@link + * SplitWatermarkExtractor}. + */ +class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final SplitWatermarkExtractor timeExtractor; + private String lastSplitId = null; + private long watermark; + + WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { + this.timeExtractor = timeExtractor; + } + + @Override + public void emitRecord( + RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { + if (!split.splitId().equals(lastSplitId)) { + long newWatermark = timeExtractor.extractWatermark(split); + if (newWatermark < watermark) { + LOG.info( + "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", + watermark, + newWatermark, + lastSplitId, + split.splitId()); + } else { + watermark = newWatermark; + output.emitWatermark(new Watermark(watermark)); + LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); + } + + lastSplitId = split.splitId(); + } + + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index 64e03d77debe..56ee92014d12 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.split; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** @@ -45,7 +46,7 @@ public static SerializableComparator fileSequenceNumber() { o1); Preconditions.checkNotNull( seq2, - "IInvalid file sequence number: null. Doesn't support splits written with V1 format: %s", + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", o2); int temp = Long.compare(seq1, seq2); @@ -56,4 +57,20 @@ public static SerializableComparator fileSequenceNumber() { } }; } + + /** Comparator which orders the splits based on watermark of the splits */ + public static SerializableComparator watermark( + SplitWatermarkExtractor watermarkExtractor) { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + long watermark1 = watermarkExtractor.extractWatermark(o1); + long watermark2 = watermarkExtractor.extractWatermark(o2); + + int temp = Long.compare(watermark1, watermark2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 70e7a79d8373..7d991ee603c9 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -39,6 +39,7 @@ import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -97,6 +98,11 @@ protected List generateRecords(int numRecords, long seed) { return RandomGenericData.generate(schema(), numRecords, seed); } + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); + } + @Test public void testBoundedWithTaskManagerFailover() throws Exception { testBoundedIcebergSource(FailoverType.TM); @@ -150,8 +156,7 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio RecordCounterToFail::continueProcessing, miniClusterResource.getMiniCluster()); - SimpleDataUtil.assertTableRecords( - sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } @Test @@ -214,8 +219,7 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - SimpleDataUtil.assertTableRecords( - sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } // ------------------------------------------------------------------------ diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java new file mode 100644 index 000000000000..f7dc931c506c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; + +public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { + // Increment ts by 15 minutes for each generateRecords batch + private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); + // Within a batch, increment ts by 1 second + private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); + + private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + + @Override + protected IcebergSource.Builder sourceBuilder() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA); + } + + @Override + protected Schema schema() { + return TestFixtures.TS_SCHEMA; + } + + @Override + protected List generateRecords(int numRecords, long seed) { + // Override the ts field to create a more realistic situation for event time alignment + tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); + return RandomGenericData.generate(schema(), numRecords, seed).stream() + .peek( + record -> { + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), + ZoneId.of("Z")); + record.setField("ts", ts); + }) + .collect(Collectors.toList()); + } + + /** + * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves + * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates + * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the + * {@link LocalDateTime} to a Long type so that Comparators can continue to work. + */ + @Override + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> { + SimpleDataUtil.equalsRecords( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + }); + } + + private List convertLocalDateTimeToMilli(List records) { + return records.stream() + .peek( + r -> { + LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); + r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java new file mode 100644 index 000000000000..0bb2eb7766e9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -0,0 +1,451 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + private static final ConcurrentMap windows = Maps.newConcurrentMap(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + *

    + *
  • - Ordering of the splits + *
  • - Emitting of watermarks + *
  • - Firing windows based on watermarks + *
+ * + *

The test generates 4 splits + * + *

    + *
  • - Split 1 - Watermark 100 min + *
  • - Split 2, 3 - Watermark 0 min + *
  • - Split 4 - Watermark 6 min + *
+ * + *

Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + *

Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + *

    + *
  • - Split 2, 3 + *
  • - Split 4 + *
  • - Split 1 + *
+ * + *

As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + *

Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction() { + @Override + public void apply( + TimeWindow window, Iterable values, Collector out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicInteger count = new AtomicInteger(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + windows.put(window.getStart(), count.get()); + } + }); + + // Use static variable to collect the windows, since other solutions were flaky + windows.clear(); + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of( + 0L, + RECORD_NUM_FOR_2_SPLITS, + TimeUnit.MINUTES.toMillis(5), + 2, + TimeUnit.MINUTES.toMillis(100), + 3))); + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + *

    + *
  • - Emitting of watermarks + *
  • - Watermark alignment + *
+ * + *

The test generates 3 splits + * + *

    + *
  • - Split 1 - Watermark 100 min + *
  • - Split 2, 3 - Watermark 0 min + *
+ * + * The splits are read in the following order: + * + *
    + *
  • - Split 2, 3 (Task Manager 1, Task Manager 2) + *
  • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + *
+ * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + *

The status of the watermark alignment is checked by the alignment related metrics. + * + *

Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + *

After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + try (CloseableIterator resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); + + // Check that the read the non-blocked data + // The first RECORD_NUM_FOR_2_SPLITS should be read + // 1 or more from the runaway reader should be arrived depending on thread scheduling + waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + List newBatch1 = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + List newBatch2 = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + dataAppender.appendToTable( + dataAppender.writeFile(newBatch1), dataAppender.writeFile(newBatch2)); + // The records received will highly depend on scheduling + // We minimally get 3 records from the non-blocked reader + // We might get 1 record from the blocked reader (as part of the previous batch - + // file_1-recordTs_100) + // We might get 3 records form the non-blocked reader if it gets both new splits + waitForRecords(resultIterator, 3); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); + + // Add some new records which should unblock the throttled reader + batch = + ImmutableList.of( + generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); + dataAppender.appendToTable(batch); + // We should get all the records at this point + waitForRecords(resultIterator, 6); + + // Wait for the new drift to decrease below the allowed drift to signal the normal state + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); + } + } + + protected IcebergSource source() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA) + .splitSize(100L) + .streaming(true) + .monitorInterval(Duration.ofMillis(2)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + } + + protected Record generateRecord(int minutes, String str) { + // Override the ts field to create a more realistic situation for event time alignment + Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), + ZoneId.of("Z")); + record.setField("ts", ts); + record.setField("str", str); + return record; + } + + protected void assertRecords( + Collection expectedRecords, CloseableIterator iterator) throws Exception { + Set expected = + expectedRecords.stream() + .map(e -> RowDataConverter.convert(TestFixtures.TS_SCHEMA, e)) + .collect(Collectors.toSet()); + Assert.assertEquals(expected, waitForRecords(iterator, expectedRecords.size())); + } + + protected Set waitForRecords(CloseableIterator iterator, int num) { + Set received = Sets.newHashSetWithExpectedSize(num); + assertThat( + CompletableFuture.supplyAsync( + () -> { + int count = 0; + while (count < num && iterator.hasNext()) { + received.add(iterator.next()); + count++; + } + + if (count < num) { + throw new IllegalStateException(String.format("Fail to get %d records.", num)); + } + + return true; + })) + .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); + + return received; + } + + private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { + String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; + return reporter.findMetrics(jobID, metricsName).values().stream() + .map(m -> (Gauge) m) + .filter(m -> m.getValue() == withValue) + .findFirst(); + } + + private GenericAppenderHelper appender() { + // We need to create multiple splits, so we need to generate parquet files with multiple offsets + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("write.parquet.page-size-bytes", "64"); + hadoopConf.set("write.parquet.row-group-size-bytes", "64"); + return new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + } + + private static RowData row(long time, long count) { + GenericRowData result = new GenericRowData(2); + result.setField(0, time); + result.setField(1, String.valueOf(count)); + return result; + } + + private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { + @Override + public long extractTimestamp(RowData element, long recordTimestamp) { + return element.getTimestamp(0, 0).getMillisecond(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index f28677ca9d6a..090b304942c6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -43,15 +43,13 @@ public void testEmptyInitialization() { @Test public void testStaticEnumeratorSequence() throws Exception { SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 1)); + assigner.onDiscoveredSplits(createSplits(4, 1, "1")); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); - assigner.onUnassignedSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1)); + assigner.onUnassignedSplits(createSplits(1, 1, "1")); assertSnapshot(assigner, 2); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -66,15 +64,12 @@ public void testContinuousEnumeratorSequence() throws Exception { SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - List splits1 = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + List splits1 = createSplits(1, 1, "1"); assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); - List splits2 = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + List splits2 = createSplits(1, 1, "1"); assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 2, 1)); + assigner.onDiscoveredSplits(createSplits(2, 1, "1")); assertSnapshot(assigner, 2); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -125,5 +120,11 @@ protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Assert.assertEquals(splitCount, stateBeforeGet.size()); } + protected List createSplits(int fileCount, int filesPerSplit, String version) + throws Exception { + return SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, fileCount, filesPerSplit, version); + } + protected abstract SplitAssigner splitAssigner(); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java index 8b9e132e0e22..e78634e6b873 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.iceberg.ContentFile; -import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; @@ -40,9 +39,7 @@ protected SplitAssigner splitAssigner() { public void testMultipleFilesInAnIcebergSplit() { SplitAssigner assigner = splitAssigner(); Assertions.assertThatThrownBy( - () -> - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2, "2")), + () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Please use 'split-open-file-cost'"); @@ -52,8 +49,7 @@ public void testMultipleFilesInAnIcebergSplit() { @Test public void testSplitSort() throws Exception { SplitAssigner assigner = splitAssigner(); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 5, 1, "2"); + List splits = createSplits(5, 1, "2"); assigner.onDiscoveredSplits(splits.subList(3, 5)); assigner.onDiscoveredSplits(splits.subList(0, 1)); @@ -76,7 +72,7 @@ public void testSerializable() { Assert.assertNotNull(comparator); } - protected void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { GetSplitResult result = assigner.getNext(null); ContentFile file = result.split().task().files().iterator().next().file(); Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java new file mode 100644 index 000000000000..e1fc63fda918 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SerializationUtil; +import org.junit.Assert; +import org.junit.Test; + +public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { + public static final Schema SCHEMA = + new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) + .createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 2, "2")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() { + SplitAssigner assigner = splitAssigner(); + + Instant now = Instant.now(); + List splits = + IntStream.range(0, 5) + .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) + .collect(Collectors.toList()); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, splits.get(0)); + assertGetNext(assigner, splits.get(1)); + assertGetNext(assigner, splits.get(2)); + assertGetNext(assigner, splits.get(3)); + assertGetNext(assigner, splits.get(4)); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = + SerializationUtil.serializeToBytes( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor( + TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { + GetSplitResult result = assigner.getNext(null); + Assert.assertEquals(result.split(), split); + } + + @Override + protected List createSplits( + int fileCount, int filesPerSplit, String version) { + return IntStream.range(0, fileCount / filesPerSplit) + .mapToObj( + splitNum -> + splitFromRecords( + IntStream.range(0, filesPerSplit) + .mapToObj( + fileNum -> + RandomGenericData.generate( + SCHEMA, 2, splitNum * filesPerSplit + fileNum)) + .collect(Collectors.toList()))) + .collect(Collectors.toList()); + } + + private IcebergSourceSplit splitFromInstant(Instant instant) { + Record record = GenericRecord.create(SCHEMA); + record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); + return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); + } + + private IcebergSourceSplit splitFromRecords(List> records) { + try { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + } catch (IOException e) { + throw new RuntimeException("Split creation exception", e); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index f9ceaf842263..2a2503ef2478 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -60,9 +60,12 @@ public static FileScanTask createFileTask( FileFormat fileFormat, FileAppenderFactory appenderFactory) throws IOException { - try (FileAppender appender = - appenderFactory.newAppender(Files.localOutput(file), fileFormat)) { + FileAppender appender = + appenderFactory.newAppender(Files.localOutput(file), fileFormat); + try { appender.addAll(records); + } finally { + appender.close(); } DataFile dataFile = @@ -71,6 +74,7 @@ public static FileScanTask createFileTask( .withFileSizeInBytes(file.length()) .withPath(file.toString()) .withFormat(fileFormat) + .withMetrics(appender.metrics()) .build(); ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..afe8a5d0152c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestColumnStatsWatermarkExtractor { + public static final Schema SCHEMA = + new Schema( + required(1, "timestamp_column", Types.TimestampType.withoutZone()), + required(2, "timestamptz_column", Types.TimestampType.withZone()), + required(3, "long_column", Types.LongType.get()), + required(4, "string_column", Types.StringType.get())); + + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + private static final List> TEST_RECORDS = + ImmutableList.of( + RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); + + private static final List> MIN_VALUES = + ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); + + private final String columnName; + + @BeforeClass + public static void updateMinValue() { + for (int i = 0; i < TEST_RECORDS.size(); ++i) { + for (Record r : TEST_RECORDS.get(i)) { + Map minValues = MIN_VALUES.get(i); + + LocalDateTime localDateTime = (LocalDateTime) r.get(0); + minValues.merge( + "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); + + OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); + minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); + + minValues.merge("long_column", (Long) r.get(2), Math::min); + } + } + } + + @Parameterized.Parameters(name = "{0}") + public static Collection data() { + return ImmutableList.of( + new Object[] {"timestamp_column"}, + new Object[] {"timestamptz_column"}, + new Object[] {"long_column"}); + } + + public TestColumnStatsWatermarkExtractor(String columnName) { + this.columnName = columnName; + } + + @Test + public void testSingle() throws IOException { + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + } + + @Test + public void testTimeUnit() throws IOException { + Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue() / 1000L, + extractor.extractWatermark(split(0))); + } + + @Test + public void testMultipleFiles() throws IOException { + Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); + IcebergSourceSplit combinedSplit = + IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + Assert.assertEquals( + MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); + Assert.assertEquals( + Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), + extractor.extractWatermark(combinedSplit)); + } + + @Test + public void testWrongColumn() { + Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); + Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); + } + + @Test + public void testEmptyStatistics() throws IOException { + Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); + + // Create an extractor for a column we do not have statistics + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(10, "missing_field"); + Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Missing statistics for column"); + } + + private IcebergSourceSplit split(int id) throws IOException { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + ImmutableList.of(TEST_RECORDS.get(id)), + TEMPORARY_FOLDER, + FileFormat.PARQUET, + APPENDER_FACTORY)); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index def4f436851b..88234c61123f 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -167,7 +167,12 @@ private IcebergSourceReader createReader( new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), new PlaintextEncryptionManager(), Collections.emptyList()); - return new IcebergSourceReader<>(readerMetrics, readerFunction, splitComparator, readerContext); + return new IcebergSourceReader<>( + SerializableRecordEmitter.defaultEmitter(), + readerMetrics, + readerFunction, + splitComparator, + readerContext); } private static class IdBasedComparator implements SerializableComparator { From c1190990760509e40dd8e10092196a569d684f1b Mon Sep 17 00:00:00 2001 From: Prabhu Joseph Date: Thu, 23 Nov 2023 16:35:44 +0530 Subject: [PATCH 59/94] Docs: Remove UNIQUE keyword as it is not supported in Flink (#9046) --- docs/flink-writes.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/flink-writes.md b/docs/flink-writes.md index e078a8286849..020a13cfb74f 100644 --- a/docs/flink-writes.md +++ b/docs/flink-writes.md @@ -69,7 +69,7 @@ Iceberg supports `UPSERT` based on the primary key when writing data into v2 tab ```sql CREATE TABLE `hive_catalog`.`default`.`sample` ( - `id` INT UNIQUE COMMENT 'unique id', + `id` INT COMMENT 'unique id', `data` STRING NOT NULL, PRIMARY KEY(`id`) NOT ENFORCED ) with ('format-version'='2', 'write.upsert.enabled'='true'); From 13fcf62b871dfec1b5ff09c1df35a5075cec328a Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 23 Nov 2023 10:12:26 -0800 Subject: [PATCH 60/94] Spark: Fix Fast forward before/after snapshot output for non-main branches (#8854) --- .../TestFastForwardBranchProcedure.java | 59 +++++++++++++++++++ .../FastForwardBranchProcedure.java | 11 ++-- 2 files changed, 66 insertions(+), 4 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java index 37d40957bc44..99bc86248519 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestFastForwardBranchProcedure.java @@ -188,4 +188,63 @@ public void testInvalidFastForwardBranchCases() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } + + @Test + public void testFastForwardNonExistingBranchCases() { + sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + table.refresh(); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.fast_forward(table => '%s', branch => '%s', to => '%s')", + catalogName, tableIdent, "non_existing_branch", SnapshotRef.MAIN_BRANCH)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch to fast-forward does not exist: non_existing_branch"); + + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.fast_forward(table => '%s', branch => '%s', to => '%s')", + catalogName, tableIdent, SnapshotRef.MAIN_BRANCH, "non_existing_branch")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref does not exist: non_existing_branch"); + } + + @Test + public void testFastForwardNonMain() { + sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + table.refresh(); + + String branch1 = "branch1"; + sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branch1); + String tableNameWithBranch1 = String.format("%s.branch_%s", tableName, branch1); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableNameWithBranch1); + table.refresh(); + Snapshot branch1Snapshot = table.snapshot(branch1); + + // Create branch2 from branch1 + String branch2 = "branch2"; + sql( + "ALTER TABLE %s CREATE BRANCH %s AS OF VERSION %d", + tableName, branch2, branch1Snapshot.snapshotId()); + String tableNameWithBranch2 = String.format("%s.branch_%s", tableName, branch2); + sql("INSERT INTO TABLE %s VALUES (3, 'c')", tableNameWithBranch2); + table.refresh(); + Snapshot branch2Snapshot = table.snapshot(branch2); + + List output = + sql( + "CALL %s.system.fast_forward('%s', '%s', '%s')", + catalogName, tableIdent, branch1, branch2); + List outputRow = Arrays.stream(output.get(0)).collect(Collectors.toList()); + assertThat(outputRow.get(0)).isEqualTo(branch1); + assertThat(outputRow.get(1)).isEqualTo(branch1Snapshot.snapshotId()); + assertThat(outputRow.get(2)).isEqualTo(branch2Snapshot.snapshotId()); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java index 459cc01c469b..83908f284b91 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/FastForwardBranchProcedure.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.procedures; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; @@ -77,12 +79,13 @@ public InternalRow[] call(InternalRow args) { return modifyIcebergTable( tableIdent, table -> { - long currentRef = table.currentSnapshot().snapshotId(); + Snapshot currentSnapshot = table.snapshot(source); + Preconditions.checkArgument( + currentSnapshot != null, "Branch to fast-forward does not exist: %s", source); table.manageSnapshots().fastForwardBranch(source, target).commit(); - long updatedRef = table.currentSnapshot().snapshotId(); - + long latest = table.snapshot(source).snapshotId(); InternalRow outputRow = - newInternalRow(UTF8String.fromString(source), currentRef, updatedRef); + newInternalRow(UTF8String.fromString(source), currentSnapshot.snapshotId(), latest); return new InternalRow[] {outputRow}; }); } From c817c8503d8570ed5b8a187471e02a0895f34598 Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Fri, 24 Nov 2023 11:16:39 +0530 Subject: [PATCH 61/94] Hive: Refactor HiveTableOperations with common code for View. (#9011) --- .../org/apache/iceberg/hive/HiveCatalog.java | 2 +- .../iceberg/hive/HiveOperationsBase.java | 184 ++++++++++++++++++ .../iceberg/hive/HiveTableOperations.java | 149 +++----------- 3 files changed, 213 insertions(+), 122 deletions(-) create mode 100644 hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java index 22f5b0b5cf37..46a8d33c48ac 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveCatalog.java @@ -235,7 +235,7 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) { try { Table table = clients.run(client -> client.getTable(fromDatabase, fromName)); - HiveTableOperations.validateTableIsIceberg(table, fullTableName(name, from)); + HiveOperationsBase.validateTableIsIceberg(table, fullTableName(name, from)); table.setDbName(toDatabase); table.setTableName(to.name()); diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java new file mode 100644 index 000000000000..ea24fe4e1133 --- /dev/null +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.hive; + +import java.util.Collections; +import java.util.Map; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.BaseMetastoreTableOperations; +import org.apache.iceberg.ClientPool; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.exceptions.NoSuchIcebergTableException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** All the HMS operations like table,view,materialized_view should implement this. */ +interface HiveOperationsBase { + + Logger LOG = LoggerFactory.getLogger(HiveOperationsBase.class); + // The max size is based on HMS backend database. For Hive versions below 2.3, the max table + // parameter size is 4000 + // characters, see https://issues.apache.org/jira/browse/HIVE-12274 + // set to 0 to not expose Iceberg metadata in HMS Table properties. + String HIVE_TABLE_PROPERTY_MAX_SIZE = "iceberg.hive.table-property-max-size"; + long HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT = 32672; + String NO_LOCK_EXPECTED_KEY = "expected_parameter_key"; + String NO_LOCK_EXPECTED_VALUE = "expected_parameter_value"; + + TableType tableType(); + + ClientPool metaClients(); + + long maxHiveTablePropertySize(); + + String database(); + + String table(); + + default Map hmsEnvContext(String metadataLocation) { + return metadataLocation == null + ? ImmutableMap.of() + : ImmutableMap.of( + NO_LOCK_EXPECTED_KEY, + BaseMetastoreTableOperations.METADATA_LOCATION_PROP, + NO_LOCK_EXPECTED_VALUE, + metadataLocation); + } + + default boolean exposeInHmsProperties() { + return maxHiveTablePropertySize() > 0; + } + + default void setSchema(TableMetadata metadata, Map parameters) { + parameters.remove(TableProperties.CURRENT_SCHEMA); + if (exposeInHmsProperties() && metadata.schema() != null) { + String schema = SchemaParser.toJson(metadata.schema()); + setField(parameters, TableProperties.CURRENT_SCHEMA, schema); + } + } + + default void setField(Map parameters, String key, String value) { + if (value.length() <= maxHiveTablePropertySize()) { + parameters.put(key, value); + } else { + LOG.warn( + "Not exposing {} in HMS since it exceeds {} characters", key, maxHiveTablePropertySize()); + } + } + + static void validateTableIsIceberg(Table table, String fullName) { + String tableType = table.getParameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP); + NoSuchIcebergTableException.check( + tableType != null + && tableType.equalsIgnoreCase(BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE), + "Not an iceberg table: %s (type=%s)", + fullName, + tableType); + } + + default void persistTable(Table hmsTable, boolean updateHiveTable, String metadataLocation) + throws TException, InterruptedException { + if (updateHiveTable) { + metaClients() + .run( + client -> { + MetastoreUtil.alterTable( + client, database(), table(), hmsTable, hmsEnvContext(metadataLocation)); + return null; + }); + } else { + metaClients() + .run( + client -> { + client.createTable(hmsTable); + return null; + }); + } + } + + static StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hiveEngineEnabled) { + + final StorageDescriptor storageDescriptor = new StorageDescriptor(); + storageDescriptor.setCols(HiveSchemaUtil.convert(metadata.schema())); + storageDescriptor.setLocation(metadata.location()); + SerDeInfo serDeInfo = new SerDeInfo(); + serDeInfo.setParameters(Maps.newHashMap()); + if (hiveEngineEnabled) { + storageDescriptor.setInputFormat("org.apache.iceberg.mr.hive.HiveIcebergInputFormat"); + storageDescriptor.setOutputFormat("org.apache.iceberg.mr.hive.HiveIcebergOutputFormat"); + serDeInfo.setSerializationLib("org.apache.iceberg.mr.hive.HiveIcebergSerDe"); + } else { + storageDescriptor.setOutputFormat("org.apache.hadoop.mapred.FileOutputFormat"); + storageDescriptor.setInputFormat("org.apache.hadoop.mapred.FileInputFormat"); + serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"); + } + storageDescriptor.setSerdeInfo(serDeInfo); + return storageDescriptor; + } + + static void cleanupMetadata(FileIO io, String commitStatus, String metadataLocation) { + try { + if (commitStatus.equalsIgnoreCase("FAILURE")) { + // If we are sure the commit failed, clean up the uncommitted metadata file + io.deleteFile(metadataLocation); + } + } catch (RuntimeException e) { + LOG.error("Failed to cleanup metadata file at {}", metadataLocation, e); + } + } + + default Table newHmsTable(String hmsTableOwner) { + Preconditions.checkNotNull(hmsTableOwner, "'hmsOwner' parameter can't be null"); + final long currentTimeMillis = System.currentTimeMillis(); + + Table newTable = + new Table( + table(), + database(), + hmsTableOwner, + (int) currentTimeMillis / 1000, + (int) currentTimeMillis / 1000, + Integer.MAX_VALUE, + null, + Collections.emptyList(), + Maps.newHashMap(), + null, + null, + tableType().name()); + + if (tableType().equals(TableType.EXTERNAL_TABLE)) { + newTable + .getParameters() + .put("EXTERNAL", "TRUE"); // using the external table type also requires this + } + + return newTable; + } +} diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java index f4b96822d42c..5e2d93a01d13 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java @@ -34,14 +34,11 @@ import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.api.InvalidObjectException; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.SerDeInfo; -import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.ClientPool; import org.apache.iceberg.PartitionSpecParser; -import org.apache.iceberg.SchemaParser; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.SortOrderParser; @@ -50,13 +47,11 @@ import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; -import org.apache.iceberg.exceptions.NoSuchIcebergTableException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.ConfigProperties; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.BiMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableBiMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -70,19 +65,12 @@ * TODO we should be able to extract some more commonalities to BaseMetastoreTableOperations to * avoid code duplication between this class and Metacat Tables. */ -public class HiveTableOperations extends BaseMetastoreTableOperations { +public class HiveTableOperations extends BaseMetastoreTableOperations + implements HiveOperationsBase { private static final Logger LOG = LoggerFactory.getLogger(HiveTableOperations.class); private static final String HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES = "iceberg.hive.metadata-refresh-max-retries"; - // the max size is based on HMS backend database. For Hive versions below 2.3, the max table - // parameter size is 4000 - // characters, see https://issues.apache.org/jira/browse/HIVE-12274 - // set to 0 to not expose Iceberg metadata in HMS Table properties. - private static final String HIVE_TABLE_PROPERTY_MAX_SIZE = "iceberg.hive.table-property-max-size"; - private static final String NO_LOCK_EXPECTED_KEY = "expected_parameter_key"; - private static final String NO_LOCK_EXPECTED_VALUE = "expected_parameter_value"; - private static final long HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT = 32672; private static final int HIVE_ICEBERG_METADATA_REFRESH_MAX_RETRIES_DEFAULT = 2; private static final BiMap ICEBERG_TO_HMS_TRANSLATION = ImmutableBiMap.of( @@ -156,7 +144,7 @@ protected void doRefresh() { String metadataLocation = null; try { Table table = metaClients.run(client -> client.getTable(database, tableName)); - validateTableIsIceberg(table, fullName); + HiveOperationsBase.validateTableIsIceberg(table, fullName); metadataLocation = table.getParameters().get(METADATA_LOCATION_PROP); @@ -207,11 +195,15 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { updateHiveTable = true; LOG.debug("Committing existing table: {}", fullName); } else { - tbl = newHmsTable(metadata); + tbl = + newHmsTable( + metadata.property(HiveCatalog.HMS_TABLE_OWNER, HiveHadoopUtil.currentUser())); LOG.debug("Committing new table: {}", fullName); } - tbl.setSd(storageDescriptor(metadata, hiveEngineEnabled)); // set to pickup any schema changes + tbl.setSd( + HiveOperationsBase.storageDescriptor( + metadata, hiveEngineEnabled)); // set to pickup any schema changes String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP); String baseMetadataLocation = base != null ? base.metadataFileLocation() : null; @@ -319,35 +311,6 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { "Committed to table {} with the new metadata location {}", fullName, newMetadataLocation); } - @VisibleForTesting - void persistTable(Table hmsTable, boolean updateHiveTable, String expectedMetadataLocation) - throws TException, InterruptedException { - if (updateHiveTable) { - metaClients.run( - client -> { - MetastoreUtil.alterTable( - client, - database, - tableName, - hmsTable, - expectedMetadataLocation != null - ? ImmutableMap.of( - NO_LOCK_EXPECTED_KEY, - METADATA_LOCATION_PROP, - NO_LOCK_EXPECTED_VALUE, - expectedMetadataLocation) - : ImmutableMap.of()); - return null; - }); - } else { - metaClients.run( - client -> { - client.createTable(hmsTable); - return null; - }); - } - } - @VisibleForTesting Table loadHmsTable() throws TException, InterruptedException { try { @@ -358,31 +321,6 @@ Table loadHmsTable() throws TException, InterruptedException { } } - private Table newHmsTable(TableMetadata metadata) { - Preconditions.checkNotNull(metadata, "'metadata' parameter can't be null"); - final long currentTimeMillis = System.currentTimeMillis(); - - Table newTable = - new Table( - tableName, - database, - metadata.property(HiveCatalog.HMS_TABLE_OWNER, HiveHadoopUtil.currentUser()), - (int) currentTimeMillis / 1000, - (int) currentTimeMillis / 1000, - Integer.MAX_VALUE, - null, - Collections.emptyList(), - Maps.newHashMap(), - null, - null, - TableType.EXTERNAL_TABLE.toString()); - - newTable - .getParameters() - .put("EXTERNAL", "TRUE"); // using the external table type also requires this - return newTable; - } - private void setHmsTableParameters( String newMetadataLocation, Table tbl, @@ -484,15 +422,6 @@ void setSnapshotSummary(Map parameters, Snapshot currentSnapshot } } - @VisibleForTesting - void setSchema(TableMetadata metadata, Map parameters) { - parameters.remove(TableProperties.CURRENT_SCHEMA); - if (exposeInHmsProperties() && metadata.schema() != null) { - String schema = SchemaParser.toJson(metadata.schema()); - setField(parameters, TableProperties.CURRENT_SCHEMA, schema); - } - } - @VisibleForTesting void setPartitionSpec(TableMetadata metadata, Map parameters) { parameters.remove(TableProperties.DEFAULT_PARTITION_SPEC); @@ -513,62 +442,40 @@ void setSortOrder(TableMetadata metadata, Map parameters) { } } - private void setField(Map parameters, String key, String value) { - if (value.length() <= maxHiveTablePropertySize) { - parameters.put(key, value); - } else { - LOG.warn( - "Not exposing {} in HMS since it exceeds {} characters", key, maxHiveTablePropertySize); - } + @Override + public long maxHiveTablePropertySize() { + return maxHiveTablePropertySize; } - private boolean exposeInHmsProperties() { - return maxHiveTablePropertySize > 0; + @Override + public String database() { + return database; } - private StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hiveEngineEnabled) { + @Override + public String table() { + return tableName; + } - final StorageDescriptor storageDescriptor = new StorageDescriptor(); - storageDescriptor.setCols(HiveSchemaUtil.convert(metadata.schema())); - storageDescriptor.setLocation(metadata.location()); - SerDeInfo serDeInfo = new SerDeInfo(); - serDeInfo.setParameters(Maps.newHashMap()); - if (hiveEngineEnabled) { - storageDescriptor.setInputFormat("org.apache.iceberg.mr.hive.HiveIcebergInputFormat"); - storageDescriptor.setOutputFormat("org.apache.iceberg.mr.hive.HiveIcebergOutputFormat"); - serDeInfo.setSerializationLib("org.apache.iceberg.mr.hive.HiveIcebergSerDe"); - } else { - storageDescriptor.setOutputFormat("org.apache.hadoop.mapred.FileOutputFormat"); - storageDescriptor.setInputFormat("org.apache.hadoop.mapred.FileInputFormat"); - serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"); - } - storageDescriptor.setSerdeInfo(serDeInfo); - return storageDescriptor; + @Override + public TableType tableType() { + return TableType.EXTERNAL_TABLE; + } + + @Override + public ClientPool metaClients() { + return metaClients; } private void cleanupMetadataAndUnlock( CommitStatus commitStatus, String metadataLocation, HiveLock lock) { try { - if (commitStatus == CommitStatus.FAILURE) { - // If we are sure the commit failed, clean up the uncommitted metadata file - io().deleteFile(metadataLocation); - } - } catch (RuntimeException e) { - LOG.error("Failed to cleanup metadata file at {}", metadataLocation, e); + HiveOperationsBase.cleanupMetadata(io(), commitStatus.name(), metadataLocation); } finally { lock.unlock(); } } - static void validateTableIsIceberg(Table table, String fullName) { - String tableType = table.getParameters().get(TABLE_TYPE_PROP); - NoSuchIcebergTableException.check( - tableType != null && tableType.equalsIgnoreCase(ICEBERG_TABLE_TYPE_VALUE), - "Not an iceberg table: %s (type=%s)", - fullName, - tableType); - } - /** * Returns if the hive engine related values should be enabled on the table, or not. * From c427a5628be6f1b5f2c4783c36c3cf7783a1916b Mon Sep 17 00:00:00 2001 From: CG Date: Fri, 24 Nov 2023 14:55:52 +0800 Subject: [PATCH 62/94] Flink: Create JUnit5 version of FlinkTestBase (#9120) --- .../flink/MiniFlinkClusterExtension.java | 53 +++++++ .../org/apache/iceberg/flink/TestBase.java | 130 ++++++++++++++++++ .../iceberg/flink/TestCatalogTableLoader.java | 21 +-- 3 files changed, 194 insertions(+), 10 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java new file mode 100644 index 000000000000..9a73b80e077d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; + +public class MiniFlinkClusterExtension { + + private static final int DEFAULT_TM_NUM = 1; + private static final int DEFAULT_PARALLELISM = 4; + + public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private MiniFlinkClusterExtension() {} + + /** + * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't + * break the unit tests because of the class loader leak issue. In our iceberg integration tests, + * there're some that will assert the results after finished the flink jobs, so actually we may + * access the class loader that has been closed by the flink task managers if we enable the switch + * classloader.check-leaked-classloader by default. + */ + public static MiniClusterExtension createWithClassloaderCheckDisabled() { + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java new file mode 100644 index 000000000000..4fc0207f269e --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public abstract class TestBase extends TestBaseUtils { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @TempDir Path temporaryDirectory; + + private static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + + private volatile TableEnvironment tEnv = null; + + @BeforeAll + public static void startMetastore() { + TestBase.metastore = new TestHiveMetastore(); + metastore.start(); + TestBase.hiveConf = metastore.hiveConf(); + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterAll + public static void stopMetastore() throws Exception { + metastore.stop(); + TestBase.catalog = null; + } + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + + TableEnvironment env = TableEnvironment.create(settings); + env.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + tEnv = env; + } + } + } + return tEnv; + } + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + Assertions.assertThat(actual) + .isNotNull() + .as(message) + .containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG default_catalog"); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java index b3a2d45261b7..147d2a173ddc 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java @@ -35,32 +35,33 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; /** Test for {@link TableLoader}. */ -public class TestCatalogTableLoader extends FlinkTestBase { +public class TestCatalogTableLoader extends TestBase { private static File warehouse = null; private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); private static final Schema SCHEMA = new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - @BeforeClass + @BeforeAll public static void createWarehouse() throws IOException { warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); + Assertions.assertThat(warehouse.delete()).isTrue(); hiveConf.set("my_key", "my_value"); } - @AfterClass + @AfterAll public static void dropWarehouse() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); + Assertions.assertThat(fs.delete(warehousePath, true)) + .as("Failed to delete " + warehousePath) + .isTrue(); } } @@ -97,7 +98,7 @@ private static void validateHadoopConf(Table table) { .as("FileIO should be a HadoopFileIO") .isInstanceOf(HadoopFileIO.class); HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); + Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); } @SuppressWarnings("unchecked") From b20d30ca04ddc19e440036b0c554f0b16ec61a1c Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Fri, 24 Nov 2023 20:30:04 +0900 Subject: [PATCH 63/94] Spark: Create base classes for migration to JUnit5 (#9129) --- .../apache/iceberg/spark/CatalogTestBase.java | 51 ++++ .../iceberg/spark/SparkTestHelperBase.java | 19 +- .../org/apache/iceberg/spark/TestBase.java | 287 ++++++++++++++++++ .../iceberg/spark/TestBaseWithCatalog.java | 131 ++++++++ .../spark/actions/TestSparkFileRewriter.java | 109 ++++--- 5 files changed, 535 insertions(+), 62 deletions(-) create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java new file mode 100644 index 000000000000..dbb839eacc48 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/CatalogTestBase.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark; + +import java.util.Map; +import java.util.stream.Stream; +import org.junit.jupiter.params.provider.Arguments; + +public abstract class CatalogTestBase extends TestBaseWithCatalog { + + // these parameters are broken out to avoid changes that need to modify lots of test suites + public static Stream parameters() { + return Stream.of( + Arguments.of( + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties()), + Arguments.of( + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties()), + Arguments.of( + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + SparkCatalogConfig.SPARK.properties())); + } + + public CatalogTestBase(SparkCatalogConfig config) { + super(config); + } + + public CatalogTestBase(String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java index 97484702cad6..6fa8da841171 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java @@ -22,7 +22,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.spark.sql.Row; -import org.junit.Assert; +import org.assertj.core.api.Assertions; public class SparkTestHelperBase { protected static final Object ANY = new Object(); @@ -55,12 +55,13 @@ private Object[] toJava(Row row) { protected void assertEquals( String context, List expectedRows, List actualRows) { - Assert.assertEquals( - context + ": number of results should match", expectedRows.size(), actualRows.size()); + Assertions.assertThat(actualRows) + .as(context + ": number of results should match") + .hasSameSizeAs(expectedRows); for (int row = 0; row < expectedRows.size(); row += 1) { Object[] expected = expectedRows.get(row); Object[] actual = actualRows.get(row); - Assert.assertEquals("Number of columns should match", expected.length, actual.length); + Assertions.assertThat(actual).as("Number of columns should match").hasSameSizeAs(expected); for (int col = 0; col < actualRows.get(row).length; col += 1) { String newContext = String.format("%s: row %d col %d", context, row + 1, col + 1); assertEquals(newContext, expected, actual); @@ -69,19 +70,23 @@ protected void assertEquals( } protected void assertEquals(String context, Object[] expectedRow, Object[] actualRow) { - Assert.assertEquals("Number of columns should match", expectedRow.length, actualRow.length); + Assertions.assertThat(actualRow) + .as("Number of columns should match") + .hasSameSizeAs(expectedRow); for (int col = 0; col < actualRow.length; col += 1) { Object expectedValue = expectedRow[col]; Object actualValue = actualRow[col]; if (expectedValue != null && expectedValue.getClass().isArray()) { String newContext = String.format("%s (nested col %d)", context, col + 1); if (expectedValue instanceof byte[]) { - Assert.assertArrayEquals(newContext, (byte[]) expectedValue, (byte[]) actualValue); + Assertions.assertThat(actualValue).as(newContext).isEqualTo(expectedValue); } else { assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue); } } else if (expectedValue != ANY) { - Assert.assertEquals(context + " contents should match", expectedValue, actualValue); + Assertions.assertThat(actualValue) + .as(context + " contents should match") + .isEqualTo(expectedValue); } } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java new file mode 100644 index 000000000000..a456fdcf4450 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBase.java @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark; + +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.net.URI; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import java.util.Map; +import java.util.TimeZone; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.execution.QueryExecution; +import org.apache.spark.sql.execution.SparkPlan; +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.util.QueryExecutionListener; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; + +public abstract class TestBase extends SparkTestHelperBase { + + protected static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static SparkSession spark = null; + protected static JavaSparkContext sparkContext = null; + protected static HiveCatalog catalog = null; + + @BeforeAll + public static void startMetastoreAndSpark() { + TestBase.metastore = new TestHiveMetastore(); + metastore.start(); + TestBase.hiveConf = metastore.hiveConf(); + + TestBase.spark = + SparkSession.builder() + .master("local[2]") + .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") + .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) + .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") + .enableHiveSupport() + .getOrCreate(); + + TestBase.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + + try { + catalog.createNamespace(Namespace.of("default")); + } catch (AlreadyExistsException ignored) { + // the default namespace already exists. ignore the create error + } + } + + @AfterAll + public static void stopMetastoreAndSpark() throws Exception { + TestBase.catalog = null; + if (metastore != null) { + metastore.stop(); + TestBase.metastore = null; + } + if (spark != null) { + spark.stop(); + TestBase.spark = null; + TestBase.sparkContext = null; + } + } + + protected long waitUntilAfter(long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); + } + return current; + } + + protected List sql(String query, Object... args) { + List rows = spark.sql(String.format(query, args)).collectAsList(); + if (rows.size() < 1) { + return ImmutableList.of(); + } + + return rowsToJava(rows); + } + + protected Object scalarSql(String query, Object... args) { + List rows = sql(query, args); + Assertions.assertThat(rows.size()).as("Scalar SQL should return one row").isEqualTo(1); + Object[] row = Iterables.getOnlyElement(rows); + Assertions.assertThat(row.length).as("Scalar SQL should return one value").isEqualTo(1); + return row[0]; + } + + protected Object[] row(Object... values) { + return values; + } + + protected static String dbPath(String dbName) { + return metastore.getDatabasePath(dbName); + } + + protected void withUnavailableFiles(Iterable> files, Action action) { + Iterable fileLocations = Iterables.transform(files, file -> file.path().toString()); + withUnavailableLocations(fileLocations, action); + } + + private void move(String location, String newLocation) { + Path path = Paths.get(URI.create(location)); + Path tempPath = Paths.get(URI.create(newLocation)); + + try { + Files.move(path, tempPath); + } catch (IOException e) { + throw new UncheckedIOException("Failed to move: " + location, e); + } + } + + protected void withUnavailableLocations(Iterable locations, Action action) { + for (String location : locations) { + move(location, location + "_temp"); + } + + try { + action.invoke(); + } finally { + for (String location : locations) { + move(location + "_temp", location); + } + } + } + + protected void withDefaultTimeZone(String zoneId, Action action) { + TimeZone currentZone = TimeZone.getDefault(); + try { + TimeZone.setDefault(TimeZone.getTimeZone(zoneId)); + action.invoke(); + } finally { + TimeZone.setDefault(currentZone); + } + } + + protected void withSQLConf(Map conf, Action action) { + SQLConf sqlConf = SQLConf.get(); + + Map currentConfValues = Maps.newHashMap(); + conf.keySet() + .forEach( + confKey -> { + if (sqlConf.contains(confKey)) { + String currentConfValue = sqlConf.getConfString(confKey); + currentConfValues.put(confKey, currentConfValue); + } + }); + + conf.forEach( + (confKey, confValue) -> { + if (SQLConf.isStaticConfigKey(confKey)) { + throw new RuntimeException("Cannot modify the value of a static config: " + confKey); + } + sqlConf.setConfString(confKey, confValue); + }); + + try { + action.invoke(); + } finally { + conf.forEach( + (confKey, confValue) -> { + if (currentConfValues.containsKey(confKey)) { + sqlConf.setConfString(confKey, currentConfValues.get(confKey)); + } else { + sqlConf.unsetConf(confKey); + } + }); + } + } + + protected Dataset jsonToDF(String schema, String... records) { + Dataset jsonDF = spark.createDataset(ImmutableList.copyOf(records), Encoders.STRING()); + return spark.read().schema(schema).json(jsonDF); + } + + protected void append(String table, String... jsonRecords) { + try { + String schema = spark.table(table).schema().toDDL(); + Dataset df = jsonToDF(schema, jsonRecords); + df.coalesce(1).writeTo(table).append(); + } catch (NoSuchTableException e) { + throw new RuntimeException("Failed to write data", e); + } + } + + protected String tablePropsAsString(Map tableProps) { + StringBuilder stringBuilder = new StringBuilder(); + + for (Map.Entry property : tableProps.entrySet()) { + if (stringBuilder.length() > 0) { + stringBuilder.append(", "); + } + stringBuilder.append(String.format("'%s' '%s'", property.getKey(), property.getValue())); + } + + return stringBuilder.toString(); + } + + protected SparkPlan executeAndKeepPlan(String query, Object... args) { + return executeAndKeepPlan(() -> sql(query, args)); + } + + protected SparkPlan executeAndKeepPlan(Action action) { + AtomicReference executedPlanRef = new AtomicReference<>(); + + QueryExecutionListener listener = + new QueryExecutionListener() { + @Override + public void onSuccess(String funcName, QueryExecution qe, long durationNs) { + executedPlanRef.set(qe.executedPlan()); + } + + @Override + public void onFailure(String funcName, QueryExecution qe, Exception exception) {} + }; + + spark.listenerManager().register(listener); + + action.invoke(); + + try { + spark.sparkContext().listenerBus().waitUntilEmpty(); + } catch (TimeoutException e) { + throw new RuntimeException("Timeout while waiting for processing events", e); + } + + SparkPlan executedPlan = executedPlanRef.get(); + if (executedPlan instanceof AdaptiveSparkPlanExec) { + return ((AdaptiveSparkPlanExec) executedPlan).executedPlan(); + } else { + return executedPlan; + } + } + + @FunctionalInterface + protected interface Action { + void invoke(); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java new file mode 100644 index 000000000000..83767039bb96 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.PlanningMode; +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.hadoop.HadoopCatalog; +import org.apache.iceberg.util.PropertyUtil; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; + +public abstract class TestBaseWithCatalog extends TestBase { + protected static File warehouse = null; + + @BeforeAll + public static void createWarehouse() throws IOException { + TestBaseWithCatalog.warehouse = File.createTempFile("warehouse", null); + Assertions.assertThat(warehouse.delete()).isTrue(); + } + + @AfterAll + public static void dropWarehouse() throws IOException { + if (warehouse != null && warehouse.exists()) { + Path warehousePath = new Path(warehouse.getAbsolutePath()); + FileSystem fs = warehousePath.getFileSystem(hiveConf); + Assertions.assertThat(fs.delete(warehousePath, true)) + .as("Failed to delete " + warehousePath) + .isTrue(); + } + } + + @TempDir protected File temp; + + protected final String catalogName; + protected final Map catalogConfig; + protected final Catalog validationCatalog; + protected final SupportsNamespaces validationNamespaceCatalog; + protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table"); + protected final String tableName; + + public TestBaseWithCatalog() { + this(SparkCatalogConfig.HADOOP); + } + + public TestBaseWithCatalog(SparkCatalogConfig config) { + this(config.catalogName(), config.implementation(), config.properties()); + } + + public TestBaseWithCatalog( + String catalogName, String implementation, Map config) { + this.catalogName = catalogName; + this.catalogConfig = config; + this.validationCatalog = + catalogName.equals("testhadoop") + ? new HadoopCatalog(spark.sessionState().newHadoopConf(), "file:" + warehouse) + : catalog; + this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + + spark.conf().set("spark.sql.catalog." + catalogName, implementation); + config.forEach( + (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + "." + key, value)); + + if (config.get("type").equalsIgnoreCase("hadoop")) { + spark.conf().set("spark.sql.catalog." + catalogName + ".warehouse", "file:" + warehouse); + } + + this.tableName = + (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default.table"; + + sql("CREATE NAMESPACE IF NOT EXISTS default"); + } + + protected String tableName(String name) { + return (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default." + name; + } + + protected String commitTarget() { + return tableName; + } + + protected String selectTarget() { + return tableName; + } + + protected boolean cachingCatalogEnabled() { + return PropertyUtil.propertyAsBoolean( + catalogConfig, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); + } + + protected void configurePlanningMode(PlanningMode planningMode) { + configurePlanningMode(tableName, planningMode); + } + + protected void configurePlanningMode(String table, PlanningMode planningMode) { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + table, + TableProperties.DATA_PLANNING_MODE, + planningMode.modeName(), + TableProperties.DELETE_PLANNING_MODE, + planningMode.modeName()); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java index 055e5be681c6..0da6bdb3622b 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java @@ -33,16 +33,15 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.spark.SparkTestBase; +import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.types.Types.IntegerType; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StringType; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; -public class TestSparkFileRewriter extends SparkTestBase { +public class TestSparkFileRewriter extends TestBase { private static final TableIdentifier TABLE_IDENT = TableIdentifier.of("default", "tbl"); private static final Schema SCHEMA = @@ -53,7 +52,7 @@ public class TestSparkFileRewriter extends SparkTestBase { PartitionSpec.builderFor(SCHEMA).identity("dep").build(); private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - @After + @AfterEach public void removeTable() { catalog.dropTable(TABLE_IDENT); } @@ -110,9 +109,9 @@ private void checkDataFileSizeFiltering(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assert.assertEquals("Must have 1 group", 1, Iterables.size(groups)); + Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assert.assertEquals("Must rewrite 2 files", 2, group.size()); + Assertions.assertThat(group).as("Must rewrite 2 files").hasSize(2); } private void checkDataFilesDeleteThreshold(SizeBasedDataRewriter rewriter) { @@ -129,9 +128,9 @@ private void checkDataFilesDeleteThreshold(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assert.assertEquals("Must have 1 group", 1, Iterables.size(groups)); + Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assert.assertEquals("Must rewrite 1 file", 1, group.size()); + Assertions.assertThat(group).as("Must rewrite 1 file").hasSize(1); } private void checkDataFileGroupWithEnoughFiles(SizeBasedDataRewriter rewriter) { @@ -152,9 +151,9 @@ private void checkDataFileGroupWithEnoughFiles(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assert.assertEquals("Must have 1 group", 1, Iterables.size(groups)); + Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assert.assertEquals("Must rewrite 4 files", 4, group.size()); + Assertions.assertThat(group).as("Must rewrite 4 files").hasSize(4); } private void checkDataFileGroupWithEnoughData(SizeBasedDataRewriter rewriter) { @@ -172,9 +171,9 @@ private void checkDataFileGroupWithEnoughData(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assert.assertEquals("Must have 1 group", 1, Iterables.size(groups)); + Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assert.assertEquals("Must rewrite 3 files", 3, group.size()); + Assertions.assertThat(group).as("Must rewrite 3 files").hasSize(3); } private void checkDataFileGroupWithTooMuchData(SizeBasedDataRewriter rewriter) { @@ -190,9 +189,9 @@ private void checkDataFileGroupWithTooMuchData(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assert.assertEquals("Must have 1 group", 1, Iterables.size(groups)); + Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assert.assertEquals("Must rewrite big file", 1, group.size()); + Assertions.assertThat(group).as("Must rewrite big file").hasSize(1); } @Test @@ -240,17 +239,17 @@ public void testBinPackDataValidOptions() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA); SparkBinPackDataRewriter rewriter = new SparkBinPackDataRewriter(spark, table); - Assert.assertEquals( - "Rewriter must report all supported options", - ImmutableSet.of( - SparkBinPackDataRewriter.TARGET_FILE_SIZE_BYTES, - SparkBinPackDataRewriter.MIN_FILE_SIZE_BYTES, - SparkBinPackDataRewriter.MAX_FILE_SIZE_BYTES, - SparkBinPackDataRewriter.MIN_INPUT_FILES, - SparkBinPackDataRewriter.REWRITE_ALL, - SparkBinPackDataRewriter.MAX_FILE_GROUP_SIZE_BYTES, - SparkBinPackDataRewriter.DELETE_FILE_THRESHOLD), - rewriter.validOptions()); + Assertions.assertThat(rewriter.validOptions()) + .as("Rewriter must report all supported options") + .isEqualTo( + ImmutableSet.of( + SparkBinPackDataRewriter.TARGET_FILE_SIZE_BYTES, + SparkBinPackDataRewriter.MIN_FILE_SIZE_BYTES, + SparkBinPackDataRewriter.MAX_FILE_SIZE_BYTES, + SparkBinPackDataRewriter.MIN_INPUT_FILES, + SparkBinPackDataRewriter.REWRITE_ALL, + SparkBinPackDataRewriter.MAX_FILE_GROUP_SIZE_BYTES, + SparkBinPackDataRewriter.DELETE_FILE_THRESHOLD)); } @Test @@ -258,19 +257,19 @@ public void testSortDataValidOptions() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA); SparkSortDataRewriter rewriter = new SparkSortDataRewriter(spark, table, SORT_ORDER); - Assert.assertEquals( - "Rewriter must report all supported options", - ImmutableSet.of( - SparkSortDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, - SparkSortDataRewriter.TARGET_FILE_SIZE_BYTES, - SparkSortDataRewriter.MIN_FILE_SIZE_BYTES, - SparkSortDataRewriter.MAX_FILE_SIZE_BYTES, - SparkSortDataRewriter.MIN_INPUT_FILES, - SparkSortDataRewriter.REWRITE_ALL, - SparkSortDataRewriter.MAX_FILE_GROUP_SIZE_BYTES, - SparkSortDataRewriter.DELETE_FILE_THRESHOLD, - SparkSortDataRewriter.COMPRESSION_FACTOR), - rewriter.validOptions()); + Assertions.assertThat(rewriter.validOptions()) + .as("Rewriter must report all supported options") + .isEqualTo( + ImmutableSet.of( + SparkSortDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, + SparkSortDataRewriter.TARGET_FILE_SIZE_BYTES, + SparkSortDataRewriter.MIN_FILE_SIZE_BYTES, + SparkSortDataRewriter.MAX_FILE_SIZE_BYTES, + SparkSortDataRewriter.MIN_INPUT_FILES, + SparkSortDataRewriter.REWRITE_ALL, + SparkSortDataRewriter.MAX_FILE_GROUP_SIZE_BYTES, + SparkSortDataRewriter.DELETE_FILE_THRESHOLD, + SparkSortDataRewriter.COMPRESSION_FACTOR)); } @Test @@ -279,21 +278,21 @@ public void testZOrderDataValidOptions() { ImmutableList zOrderCols = ImmutableList.of("id"); SparkZOrderDataRewriter rewriter = new SparkZOrderDataRewriter(spark, table, zOrderCols); - Assert.assertEquals( - "Rewriter must report all supported options", - ImmutableSet.of( - SparkZOrderDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, - SparkZOrderDataRewriter.TARGET_FILE_SIZE_BYTES, - SparkZOrderDataRewriter.MIN_FILE_SIZE_BYTES, - SparkZOrderDataRewriter.MAX_FILE_SIZE_BYTES, - SparkZOrderDataRewriter.MIN_INPUT_FILES, - SparkZOrderDataRewriter.REWRITE_ALL, - SparkZOrderDataRewriter.MAX_FILE_GROUP_SIZE_BYTES, - SparkZOrderDataRewriter.DELETE_FILE_THRESHOLD, - SparkZOrderDataRewriter.COMPRESSION_FACTOR, - SparkZOrderDataRewriter.MAX_OUTPUT_SIZE, - SparkZOrderDataRewriter.VAR_LENGTH_CONTRIBUTION), - rewriter.validOptions()); + Assertions.assertThat(rewriter.validOptions()) + .as("Rewriter must report all supported options") + .isEqualTo( + ImmutableSet.of( + SparkZOrderDataRewriter.SHUFFLE_PARTITIONS_PER_FILE, + SparkZOrderDataRewriter.TARGET_FILE_SIZE_BYTES, + SparkZOrderDataRewriter.MIN_FILE_SIZE_BYTES, + SparkZOrderDataRewriter.MAX_FILE_SIZE_BYTES, + SparkZOrderDataRewriter.MIN_INPUT_FILES, + SparkZOrderDataRewriter.REWRITE_ALL, + SparkZOrderDataRewriter.MAX_FILE_GROUP_SIZE_BYTES, + SparkZOrderDataRewriter.DELETE_FILE_THRESHOLD, + SparkZOrderDataRewriter.COMPRESSION_FACTOR, + SparkZOrderDataRewriter.MAX_OUTPUT_SIZE, + SparkZOrderDataRewriter.VAR_LENGTH_CONTRIBUTION)); } @Test From 10a856e0f4e6bd1653ed66a5ba232277834b9a8d Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 24 Nov 2023 17:57:04 +0100 Subject: [PATCH 64/94] Flink: Proper backport for #8852 (#9146) --- .../flink/source/TestIcebergSourceFailover.java | 9 +++++---- .../source/TestStreamingMonitorFunction.java | 16 ++++++---------- .../flink/source/TestIcebergSourceFailover.java | 9 +++++---- .../source/TestStreamingMonitorFunction.java | 15 +++++---------- 4 files changed, 21 insertions(+), 28 deletions(-) diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7186db21f709..70e7a79d8373 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -18,8 +18,6 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.SimpleDataUtil.assertTableRecords; - import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -46,6 +44,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkSink; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; @@ -151,7 +150,8 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio RecordCounterToFail::continueProcessing, miniClusterResource.getMiniCluster()); - assertTableRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + SimpleDataUtil.assertTableRecords( + sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } @Test @@ -214,7 +214,8 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - assertTableRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + SimpleDataUtil.assertTableRecords( + sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } // ------------------------------------------------------------------------ diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 132bbd7b2a9e..8af1dd883f4c 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -109,8 +109,7 @@ public void testConsumeWithoutStartSnapshotId() throws Exception { harness.setup(); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, function); awaitExpectedSplits(sourceContext); @@ -143,8 +142,7 @@ public void testConsumeFromStartSnapshotId() throws Exception { harness.setup(); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, function); awaitExpectedSplits(sourceContext); @@ -176,11 +174,11 @@ public void testConsumeFromStartTag() throws Exception { harness.setup(); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, function); awaitExpectedSplits(sourceContext); + // Stop the stream task. function.close(); @@ -200,8 +198,7 @@ public void testCheckpointRestore() throws Exception { harness.setup(); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, func); awaitExpectedSplits(sourceContext); @@ -223,8 +220,7 @@ public void testCheckpointRestore() throws Exception { harness.initializeState(state); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, newFunc); awaitExpectedSplits(sourceContext); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7186db21f709..70e7a79d8373 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -18,8 +18,6 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.SimpleDataUtil.assertTableRecords; - import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -46,6 +44,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkSink; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; @@ -151,7 +150,8 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio RecordCounterToFail::continueProcessing, miniClusterResource.getMiniCluster()); - assertTableRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + SimpleDataUtil.assertTableRecords( + sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } @Test @@ -214,7 +214,8 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - assertTableRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + SimpleDataUtil.assertTableRecords( + sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } // ------------------------------------------------------------------------ diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 0c3f54cc726a..6d1891baf538 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -109,8 +109,7 @@ public void testConsumeWithoutStartSnapshotId() throws Exception { harness.setup(); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, function); awaitExpectedSplits(sourceContext); @@ -143,8 +142,7 @@ public void testConsumeFromStartSnapshotId() throws Exception { harness.setup(); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, function); awaitExpectedSplits(sourceContext); @@ -176,8 +174,7 @@ public void testConsumeFromStartTag() throws Exception { harness.setup(); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, function); awaitExpectedSplits(sourceContext); @@ -201,8 +198,7 @@ public void testCheckpointRestore() throws Exception { harness.setup(); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, func); awaitExpectedSplits(sourceContext); @@ -224,8 +220,7 @@ public void testCheckpointRestore() throws Exception { harness.initializeState(state); harness.open(); - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); runSourceFunctionInTask(sourceContext, newFunc); awaitExpectedSplits(sourceContext); From 1a073ddc47d53c005adcbb35399da6255123b652 Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 24 Nov 2023 20:00:29 +0100 Subject: [PATCH 65/94] Flink: Backport #8803 to v1.16 and v1.15 (#9144) Co-authored-by: Peter Vary --- .../flink/source/FlinkSplitPlanner.java | 4 + .../iceberg/flink/source/ScanContext.java | 17 +++ .../TestContinuousSplitPlannerImpl.java | 143 ++++++++++++++++-- .../flink/source/FlinkSplitPlanner.java | 4 + .../iceberg/flink/source/ScanContext.java | 17 +++ .../TestContinuousSplitPlannerImpl.java | 143 ++++++++++++++++-- 6 files changed, 304 insertions(+), 24 deletions(-) diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java index ea317e93d8ba..15078809714f 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java @@ -165,6 +165,10 @@ private static > T refineScanW refinedScan = refinedScan.includeColumnStats(); } + if (context.includeStatsForColumns() != null) { + refinedScan = refinedScan.includeColumnStats(context.includeStatsForColumns()); + } + refinedScan = refinedScan.option(TableProperties.SPLIT_SIZE, context.splitSize().toString()); refinedScan = diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java index e380204e871f..4357b1f57df6 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.time.Duration; +import java.util.Collection; import java.util.List; import java.util.Map; import org.apache.flink.annotation.Internal; @@ -62,6 +63,7 @@ public class ScanContext implements Serializable { private final List filters; private final long limit; private final boolean includeColumnStats; + private final Collection includeStatsForColumns; private final Integer planParallelism; private final int maxPlanningSnapshotCount; private final int maxAllowedPlanningFailures; @@ -84,6 +86,7 @@ private ScanContext( List filters, long limit, boolean includeColumnStats, + Collection includeStatsForColumns, boolean exposeLocality, Integer planParallelism, int maxPlanningSnapshotCount, @@ -114,6 +117,7 @@ private ScanContext( this.filters = filters; this.limit = limit; this.includeColumnStats = includeColumnStats; + this.includeStatsForColumns = includeStatsForColumns; this.exposeLocality = exposeLocality; this.planParallelism = planParallelism; this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; @@ -248,6 +252,10 @@ public boolean includeColumnStats() { return includeColumnStats; } + public Collection includeStatsForColumns() { + return includeStatsForColumns; + } + public boolean exposeLocality() { return exposeLocality; } @@ -285,6 +293,7 @@ public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSn .filters(filters) .limit(limit) .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) .exposeLocality(exposeLocality) .planParallelism(planParallelism) .maxPlanningSnapshotCount(maxPlanningSnapshotCount) @@ -313,6 +322,7 @@ public ScanContext copyWithSnapshotId(long newSnapshotId) { .filters(filters) .limit(limit) .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) .exposeLocality(exposeLocality) .planParallelism(planParallelism) .maxPlanningSnapshotCount(maxPlanningSnapshotCount) @@ -349,6 +359,7 @@ public static class Builder { private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); private boolean includeColumnStats = FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); + private Collection includeStatsForColumns = null; private boolean exposeLocality; private Integer planParallelism = FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); @@ -464,6 +475,11 @@ public Builder includeColumnStats(boolean newIncludeColumnStats) { return this; } + public Builder includeColumnStats(Collection newIncludeStatsForColumns) { + this.includeStatsForColumns = newIncludeStatsForColumns; + return this; + } + public Builder exposeLocality(boolean newExposeLocality) { this.exposeLocality = newExposeLocality; return this; @@ -531,6 +547,7 @@ public ScanContext build() { filters, limit, includeColumnStats, + includeStatsForColumns, exposeLocality, planParallelism, maxPlanningSnapshotCount, diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 85174b4ab273..bb747caae589 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -84,7 +84,7 @@ private void appendTwoSnapshots() throws IOException { } /** @return the last enumerated snapshot id */ - private IcebergEnumeratorPosition verifyOneCycle( + private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { List batch = @@ -106,7 +106,7 @@ private IcebergEnumeratorPosition verifyOneCycle( Assert.assertEquals( dataFile.path().toString(), Iterables.getOnlyElement(split.task().files()).file().path().toString()); - return result.toPosition(); + return new CycleResult(result.toPosition(), split); } @Test @@ -135,7 +135,7 @@ public void testTableScanThenIncrementalWithEmptyTable() throws Exception { // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -169,7 +169,7 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -206,7 +206,7 @@ public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { // next 3 snapshots IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -251,7 +251,7 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -281,7 +281,7 @@ public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -323,12 +323,12 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @Test - public void testIncrementalFromSnapshotIdWithEmptyTable() throws Exception { + public void testIncrementalFromSnapshotIdWithEmptyTable() { ScanContext scanContextWithInvalidSnapshotId = ScanContext.builder() .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) @@ -409,12 +409,12 @@ public void testIncrementalFromSnapshotId() throws Exception { IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @Test - public void testIncrementalFromSnapshotTimestampWithEmptyTable() throws Exception { + public void testIncrementalFromSnapshotTimestampWithEmptyTable() { ScanContext scanContextWithInvalidSnapshotId = ScanContext.builder() .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) @@ -489,7 +489,7 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -529,6 +529,115 @@ public void testMaxPlanningSnapshotCount() throws Exception { thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); } + @Test + public void testTableScanNoStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(false) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 0); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 0); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanAllStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 3); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 3); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanSingleStat() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(ImmutableSet.of("data")) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 1); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 1); + lastPosition = result.lastPosition; + } + } + + private void verifyStatCount(IcebergSourceSplit split, int expected) { + if (expected == 0) { + split + .task() + .files() + .forEach( + f -> { + Assert.assertNull(f.file().valueCounts()); + Assert.assertNull(f.file().columnSizes()); + Assert.assertNull(f.file().lowerBounds()); + Assert.assertNull(f.file().upperBounds()); + Assert.assertNull(f.file().nanValueCounts()); + Assert.assertNull(f.file().nullValueCounts()); + }); + } else { + split + .task() + .files() + .forEach( + f -> { + Assert.assertEquals(expected, f.file().valueCounts().size()); + Assert.assertEquals(expected, f.file().columnSizes().size()); + Assert.assertEquals(expected, f.file().lowerBounds().size()); + Assert.assertEquals(expected, f.file().upperBounds().size()); + Assert.assertEquals(expected, f.file().nullValueCounts().size()); + // The nanValue is not counted for long and string fields + Assert.assertEquals(0, f.file().nanValueCounts().size()); + }); + } + } + private void verifyMaxPlanningSnapshotCountResult( ContinuousEnumerationResult result, Snapshot fromSnapshotExclusive, @@ -566,4 +675,14 @@ private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { dataAppender.appendToTable(dataFile); return tableResource.table().currentSnapshot(); } + + private static class CycleResult { + IcebergEnumeratorPosition lastPosition; + IcebergSourceSplit split; + + CycleResult(IcebergEnumeratorPosition lastPosition, IcebergSourceSplit split) { + this.lastPosition = lastPosition; + this.split = split; + } + } } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java index ea317e93d8ba..15078809714f 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java @@ -165,6 +165,10 @@ private static > T refineScanW refinedScan = refinedScan.includeColumnStats(); } + if (context.includeStatsForColumns() != null) { + refinedScan = refinedScan.includeColumnStats(context.includeStatsForColumns()); + } + refinedScan = refinedScan.option(TableProperties.SPLIT_SIZE, context.splitSize().toString()); refinedScan = diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java index e380204e871f..4357b1f57df6 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.time.Duration; +import java.util.Collection; import java.util.List; import java.util.Map; import org.apache.flink.annotation.Internal; @@ -62,6 +63,7 @@ public class ScanContext implements Serializable { private final List filters; private final long limit; private final boolean includeColumnStats; + private final Collection includeStatsForColumns; private final Integer planParallelism; private final int maxPlanningSnapshotCount; private final int maxAllowedPlanningFailures; @@ -84,6 +86,7 @@ private ScanContext( List filters, long limit, boolean includeColumnStats, + Collection includeStatsForColumns, boolean exposeLocality, Integer planParallelism, int maxPlanningSnapshotCount, @@ -114,6 +117,7 @@ private ScanContext( this.filters = filters; this.limit = limit; this.includeColumnStats = includeColumnStats; + this.includeStatsForColumns = includeStatsForColumns; this.exposeLocality = exposeLocality; this.planParallelism = planParallelism; this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; @@ -248,6 +252,10 @@ public boolean includeColumnStats() { return includeColumnStats; } + public Collection includeStatsForColumns() { + return includeStatsForColumns; + } + public boolean exposeLocality() { return exposeLocality; } @@ -285,6 +293,7 @@ public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSn .filters(filters) .limit(limit) .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) .exposeLocality(exposeLocality) .planParallelism(planParallelism) .maxPlanningSnapshotCount(maxPlanningSnapshotCount) @@ -313,6 +322,7 @@ public ScanContext copyWithSnapshotId(long newSnapshotId) { .filters(filters) .limit(limit) .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) .exposeLocality(exposeLocality) .planParallelism(planParallelism) .maxPlanningSnapshotCount(maxPlanningSnapshotCount) @@ -349,6 +359,7 @@ public static class Builder { private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); private boolean includeColumnStats = FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); + private Collection includeStatsForColumns = null; private boolean exposeLocality; private Integer planParallelism = FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); @@ -464,6 +475,11 @@ public Builder includeColumnStats(boolean newIncludeColumnStats) { return this; } + public Builder includeColumnStats(Collection newIncludeStatsForColumns) { + this.includeStatsForColumns = newIncludeStatsForColumns; + return this; + } + public Builder exposeLocality(boolean newExposeLocality) { this.exposeLocality = newExposeLocality; return this; @@ -531,6 +547,7 @@ public ScanContext build() { filters, limit, includeColumnStats, + includeStatsForColumns, exposeLocality, planParallelism, maxPlanningSnapshotCount, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 85174b4ab273..bb747caae589 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -84,7 +84,7 @@ private void appendTwoSnapshots() throws IOException { } /** @return the last enumerated snapshot id */ - private IcebergEnumeratorPosition verifyOneCycle( + private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { List batch = @@ -106,7 +106,7 @@ private IcebergEnumeratorPosition verifyOneCycle( Assert.assertEquals( dataFile.path().toString(), Iterables.getOnlyElement(split.task().files()).file().path().toString()); - return result.toPosition(); + return new CycleResult(result.toPosition(), split); } @Test @@ -135,7 +135,7 @@ public void testTableScanThenIncrementalWithEmptyTable() throws Exception { // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -169,7 +169,7 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -206,7 +206,7 @@ public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { // next 3 snapshots IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -251,7 +251,7 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -281,7 +281,7 @@ public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -323,12 +323,12 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @Test - public void testIncrementalFromSnapshotIdWithEmptyTable() throws Exception { + public void testIncrementalFromSnapshotIdWithEmptyTable() { ScanContext scanContextWithInvalidSnapshotId = ScanContext.builder() .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) @@ -409,12 +409,12 @@ public void testIncrementalFromSnapshotId() throws Exception { IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @Test - public void testIncrementalFromSnapshotTimestampWithEmptyTable() throws Exception { + public void testIncrementalFromSnapshotTimestampWithEmptyTable() { ScanContext scanContextWithInvalidSnapshotId = ScanContext.builder() .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) @@ -489,7 +489,7 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition); + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; } } @@ -529,6 +529,115 @@ public void testMaxPlanningSnapshotCount() throws Exception { thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); } + @Test + public void testTableScanNoStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(false) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 0); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 0); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanAllStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 3); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 3); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanSingleStat() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(ImmutableSet.of("data")) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 1); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 1); + lastPosition = result.lastPosition; + } + } + + private void verifyStatCount(IcebergSourceSplit split, int expected) { + if (expected == 0) { + split + .task() + .files() + .forEach( + f -> { + Assert.assertNull(f.file().valueCounts()); + Assert.assertNull(f.file().columnSizes()); + Assert.assertNull(f.file().lowerBounds()); + Assert.assertNull(f.file().upperBounds()); + Assert.assertNull(f.file().nanValueCounts()); + Assert.assertNull(f.file().nullValueCounts()); + }); + } else { + split + .task() + .files() + .forEach( + f -> { + Assert.assertEquals(expected, f.file().valueCounts().size()); + Assert.assertEquals(expected, f.file().columnSizes().size()); + Assert.assertEquals(expected, f.file().lowerBounds().size()); + Assert.assertEquals(expected, f.file().upperBounds().size()); + Assert.assertEquals(expected, f.file().nullValueCounts().size()); + // The nanValue is not counted for long and string fields + Assert.assertEquals(0, f.file().nanValueCounts().size()); + }); + } + } + private void verifyMaxPlanningSnapshotCountResult( ContinuousEnumerationResult result, Snapshot fromSnapshotExclusive, @@ -566,4 +675,14 @@ private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { dataAppender.appendToTable(dataFile); return tableResource.table().currentSnapshot(); } + + private static class CycleResult { + IcebergEnumeratorPosition lastPosition; + IcebergSourceSplit split; + + CycleResult(IcebergEnumeratorPosition lastPosition, IcebergSourceSplit split) { + this.lastPosition = lastPosition; + this.split = split; + } + } } From f246614ed9d98f84eae13ff57768d377a95d3ced Mon Sep 17 00:00:00 2001 From: CG Date: Mon, 27 Nov 2023 18:16:19 +0800 Subject: [PATCH 66/94] Flink: Backport #9078 to v1.16 and v1.15 (#9151) --- .../flink/MiniFlinkClusterExtension.java | 53 +++++++ .../org/apache/iceberg/flink/TestBase.java | 130 ++++++++++++++++++ .../iceberg/flink/TestCatalogTableLoader.java | 21 +-- .../flink/MiniFlinkClusterExtension.java | 53 +++++++ .../org/apache/iceberg/flink/TestBase.java | 130 ++++++++++++++++++ .../iceberg/flink/TestCatalogTableLoader.java | 21 +-- 6 files changed, 388 insertions(+), 20 deletions(-) create mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java create mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestBase.java diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java new file mode 100644 index 000000000000..9a73b80e077d --- /dev/null +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; + +public class MiniFlinkClusterExtension { + + private static final int DEFAULT_TM_NUM = 1; + private static final int DEFAULT_PARALLELISM = 4; + + public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private MiniFlinkClusterExtension() {} + + /** + * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't + * break the unit tests because of the class loader leak issue. In our iceberg integration tests, + * there're some that will assert the results after finished the flink jobs, so actually we may + * access the class loader that has been closed by the flink task managers if we enable the switch + * classloader.check-leaked-classloader by default. + */ + public static MiniClusterExtension createWithClassloaderCheckDisabled() { + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java new file mode 100644 index 000000000000..4fc0207f269e --- /dev/null +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public abstract class TestBase extends TestBaseUtils { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @TempDir Path temporaryDirectory; + + private static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + + private volatile TableEnvironment tEnv = null; + + @BeforeAll + public static void startMetastore() { + TestBase.metastore = new TestHiveMetastore(); + metastore.start(); + TestBase.hiveConf = metastore.hiveConf(); + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterAll + public static void stopMetastore() throws Exception { + metastore.stop(); + TestBase.catalog = null; + } + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + + TableEnvironment env = TableEnvironment.create(settings); + env.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + tEnv = env; + } + } + } + return tEnv; + } + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + Assertions.assertThat(actual) + .isNotNull() + .as(message) + .containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG default_catalog"); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java index b3a2d45261b7..147d2a173ddc 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java @@ -35,32 +35,33 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; /** Test for {@link TableLoader}. */ -public class TestCatalogTableLoader extends FlinkTestBase { +public class TestCatalogTableLoader extends TestBase { private static File warehouse = null; private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); private static final Schema SCHEMA = new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - @BeforeClass + @BeforeAll public static void createWarehouse() throws IOException { warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); + Assertions.assertThat(warehouse.delete()).isTrue(); hiveConf.set("my_key", "my_value"); } - @AfterClass + @AfterAll public static void dropWarehouse() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); + Assertions.assertThat(fs.delete(warehousePath, true)) + .as("Failed to delete " + warehousePath) + .isTrue(); } } @@ -97,7 +98,7 @@ private static void validateHadoopConf(Table table) { .as("FileIO should be a HadoopFileIO") .isInstanceOf(HadoopFileIO.class); HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); + Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); } @SuppressWarnings("unchecked") diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java new file mode 100644 index 000000000000..9a73b80e077d --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; + +public class MiniFlinkClusterExtension { + + private static final int DEFAULT_TM_NUM = 1; + private static final int DEFAULT_PARALLELISM = 4; + + public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private MiniFlinkClusterExtension() {} + + /** + * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't + * break the unit tests because of the class loader leak issue. In our iceberg integration tests, + * there're some that will assert the results after finished the flink jobs, so actually we may + * access the class loader that has been closed by the flink task managers if we enable the switch + * classloader.check-leaked-classloader by default. + */ + public static MiniClusterExtension createWithClassloaderCheckDisabled() { + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestBase.java new file mode 100644 index 000000000000..4fc0207f269e --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public abstract class TestBase extends TestBaseUtils { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @TempDir Path temporaryDirectory; + + private static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + + private volatile TableEnvironment tEnv = null; + + @BeforeAll + public static void startMetastore() { + TestBase.metastore = new TestHiveMetastore(); + metastore.start(); + TestBase.hiveConf = metastore.hiveConf(); + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterAll + public static void stopMetastore() throws Exception { + metastore.stop(); + TestBase.catalog = null; + } + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + + TableEnvironment env = TableEnvironment.create(settings); + env.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + tEnv = env; + } + } + } + return tEnv; + } + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + Assertions.assertThat(actual) + .isNotNull() + .as(message) + .containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG default_catalog"); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java index b3a2d45261b7..147d2a173ddc 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java @@ -35,32 +35,33 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; /** Test for {@link TableLoader}. */ -public class TestCatalogTableLoader extends FlinkTestBase { +public class TestCatalogTableLoader extends TestBase { private static File warehouse = null; private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); private static final Schema SCHEMA = new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - @BeforeClass + @BeforeAll public static void createWarehouse() throws IOException { warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); + Assertions.assertThat(warehouse.delete()).isTrue(); hiveConf.set("my_key", "my_value"); } - @AfterClass + @AfterAll public static void dropWarehouse() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); + Assertions.assertThat(fs.delete(warehousePath, true)) + .as("Failed to delete " + warehousePath) + .isTrue(); } } @@ -97,7 +98,7 @@ private static void validateHadoopConf(Table table) { .as("FileIO should be a HadoopFileIO") .isInstanceOf(HadoopFileIO.class); HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); + Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); } @SuppressWarnings("unchecked") From 6fc5be738d317eb1e8a5b525f5b8e40841dee678 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 27 Nov 2023 11:24:39 -0800 Subject: [PATCH 67/94] API, Core: Fix naming in fastForwardBranch/replaceBranch APIs (#9134) --- .../org/apache/iceberg/ManageSnapshots.java | 22 ++++++------ .../org/apache/iceberg/SnapshotManager.java | 8 ++--- .../UpdateSnapshotReferencesOperation.java | 35 ++++++++++--------- .../apache/iceberg/TestSnapshotManager.java | 28 ++++++++++++--- 4 files changed, 57 insertions(+), 36 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/ManageSnapshots.java b/api/src/main/java/org/apache/iceberg/ManageSnapshots.java index 2fa60472da5e..986bbb6f5809 100644 --- a/api/src/main/java/org/apache/iceberg/ManageSnapshots.java +++ b/api/src/main/java/org/apache/iceberg/ManageSnapshots.java @@ -163,26 +163,26 @@ default ManageSnapshots createBranch(String name) { ManageSnapshots replaceBranch(String name, long snapshotId); /** - * Replaces the branch with the given name to point to the source snapshot. The source branch will - * remain unchanged, the target branch will retain its retention properties. + * Replaces the {@code from} branch to point to the {@code to} snapshot. The {@code to} will + * remain unchanged, and {@code from} branch will retain its retention properties. * - * @param name Branch to replace - * @param source Source reference for the target to be replaced with + * @param from Branch to replace + * @param to The branch {@code from} should be replaced with * @return this for method chaining */ - ManageSnapshots replaceBranch(String name, String source); + ManageSnapshots replaceBranch(String from, String to); /** - * Performs a fast-forward of the given target branch up to the source snapshot if target is an - * ancestor of source. The source branch will remain unchanged, the target branch will retain its + * Performs a fast-forward of {@code from} up to the {@code to} snapshot if {@code from} is an + * ancestor of {@code to}. The {@code to} will remain unchanged, and {@code from} will retain its * retention properties. * - * @param name Branch to fast-forward - * @param source Source reference for the target to be fast forwarded to + * @param from Branch to fast-forward + * @param to Ref for the {@code from} branch to be fast forwarded to * @return this for method chaining - * @throws IllegalArgumentException if the target branch is not an ancestor of source + * @throws IllegalArgumentException if {@code from} is not an ancestor of {@code to} */ - ManageSnapshots fastForwardBranch(String name, String source); + ManageSnapshots fastForwardBranch(String from, String to); /** * Updates the minimum number of snapshots to keep for a branch. diff --git a/core/src/main/java/org/apache/iceberg/SnapshotManager.java b/core/src/main/java/org/apache/iceberg/SnapshotManager.java index c9774f3b929b..bb7ca4b11c11 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotManager.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotManager.java @@ -137,14 +137,14 @@ public ManageSnapshots replaceBranch(String name, long snapshotId) { } @Override - public ManageSnapshots replaceBranch(String name, String source) { - updateSnapshotReferencesOperation().replaceBranch(name, source); + public ManageSnapshots replaceBranch(String from, String to) { + updateSnapshotReferencesOperation().replaceBranch(from, to); return this; } @Override - public ManageSnapshots fastForwardBranch(String name, String source) { - updateSnapshotReferencesOperation().fastForward(name, source); + public ManageSnapshots fastForwardBranch(String from, String to) { + updateSnapshotReferencesOperation().fastForward(from, to); return this; } diff --git a/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java b/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java index b87bac2f014f..2c3c6c1f7e10 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java +++ b/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java @@ -106,40 +106,41 @@ public UpdateSnapshotReferencesOperation replaceBranch(String name, long snapsho return this; } - public UpdateSnapshotReferencesOperation replaceBranch(String name, String source) { - return replaceBranch(name, source, false); + public UpdateSnapshotReferencesOperation replaceBranch(String from, String to) { + return replaceBranch(from, to, false); } - public UpdateSnapshotReferencesOperation fastForward(String name, String source) { - return replaceBranch(name, source, true); + public UpdateSnapshotReferencesOperation fastForward(String from, String to) { + return replaceBranch(from, to, true); } private UpdateSnapshotReferencesOperation replaceBranch( - String name, String source, boolean fastForward) { - Preconditions.checkNotNull(name, "Target branch cannot be null"); - Preconditions.checkNotNull(source, "Source ref cannot be null"); - SnapshotRef sourceRef = updatedRefs.get(source); - SnapshotRef refToUpdate = updatedRefs.get(name); - Preconditions.checkArgument(refToUpdate != null, "Target branch does not exist: %s", name); - Preconditions.checkArgument(sourceRef != null, "Ref does not exist: %s", source); - Preconditions.checkArgument(refToUpdate.isBranch(), "Ref %s is a tag not a branch", name); + String from, String to, boolean fastForward) { + Preconditions.checkNotNull(from, "Branch to update cannot be null"); + Preconditions.checkNotNull(to, "Destination ref cannot be null"); + SnapshotRef branchToUpdate = updatedRefs.get(from); + SnapshotRef toRef = updatedRefs.get(to); + Preconditions.checkArgument( + branchToUpdate != null, "Branch to update does not exist: %s", from); + Preconditions.checkArgument(toRef != null, "Ref does not exist: %s", to); + Preconditions.checkArgument(branchToUpdate.isBranch(), "Ref %s is a tag not a branch", from); // Nothing to replace - if (sourceRef.snapshotId() == refToUpdate.snapshotId()) { + if (toRef.snapshotId() == branchToUpdate.snapshotId()) { return this; } - SnapshotRef updatedRef = SnapshotRef.builderFrom(refToUpdate, sourceRef.snapshotId()).build(); + SnapshotRef updatedRef = SnapshotRef.builderFrom(branchToUpdate, toRef.snapshotId()).build(); if (fastForward) { boolean targetIsAncestor = SnapshotUtil.isAncestorOf( - sourceRef.snapshotId(), refToUpdate.snapshotId(), base::snapshot); + toRef.snapshotId(), branchToUpdate.snapshotId(), base::snapshot); Preconditions.checkArgument( - targetIsAncestor, "Cannot fast-forward: %s is not an ancestor of %s", name, source); + targetIsAncestor, "Cannot fast-forward: %s is not an ancestor of %s", from, to); } - updatedRefs.put(name, updatedRef); + updatedRefs.put(from, updatedRef); return this; } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java index d497dbd360a7..d561d697d3e9 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java @@ -409,15 +409,15 @@ public void testReplaceBranch() { } @Test - public void testReplaceBranchNonExistingTargetBranchFails() { + public void testReplaceBranchNonExistingBranchToUpdateFails() { Assertions.assertThatThrownBy( () -> table.manageSnapshots().replaceBranch("non-existing", "other-branch").commit()) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Target branch does not exist: non-existing"); + .hasMessage("Branch to update does not exist: non-existing"); } @Test - public void testReplaceBranchNonExistingSourceFails() { + public void testReplaceBranchNonExistingToBranchFails() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("branch1", snapshotId).commit(); @@ -427,6 +427,26 @@ public void testReplaceBranchNonExistingSourceFails() { .hasMessage("Ref does not exist: non-existing"); } + @Test + public void testFastForwardBranchNonExistingFromBranchFails() { + Assertions.assertThatThrownBy( + () -> + table.manageSnapshots().fastForwardBranch("non-existing", "other-branch").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Branch to update does not exist: non-existing"); + } + + @Test + public void testFastForwardBranchNonExistingToFails() { + table.newAppend().appendFile(FILE_A).commit(); + long snapshotId = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createBranch("branch1", snapshotId).commit(); + Assertions.assertThatThrownBy( + () -> table.manageSnapshots().fastForwardBranch("branch1", "non-existing").commit()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Ref does not exist: non-existing"); + } + @Test public void testFastForward() { table.newAppend().appendFile(FILE_A).commit(); @@ -445,7 +465,7 @@ public void testFastForward() { } @Test - public void testFastForwardWhenTargetIsNotAncestorFails() { + public void testFastForwardWhenFromIsNotAncestorFails() { table.newAppend().appendFile(FILE_A).commit(); table.newAppend().appendFile(FILE_B).set("wap.id", "123456789").stageOnly().commit(); From 4e62b58f04aba4ccc2a2a846494b478d3b03f58f Mon Sep 17 00:00:00 2001 From: Andre Luis Anastacio Date: Mon, 27 Nov 2023 21:41:32 -0300 Subject: [PATCH 68/94] AWS, Core, Dell, Spark: Use Strings to verify null and empty string (#9090) * AWS: Use !Strings.isNullOrEmpty to verify null and empty string * Core: Use !Strings.isNullOrEmpty to verify null and empty string * Dell: Use !Strings.isNullOrEmpty to verify null and empty string * Spark 3.2: Use !.isEmpty() instead of .length() > 0 * Spark 3.3: Use !.isEmpty() instead of .length() > 0 --- .../org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java | 3 ++- .../main/java/org/apache/iceberg/aws/glue/GlueCatalog.java | 6 +++--- .../main/java/org/apache/iceberg/hadoop/HadoopCatalog.java | 3 ++- core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java | 3 ++- .../src/main/java/org/apache/iceberg/util/LocationUtil.java | 4 ++-- .../main/java/org/apache/iceberg/dell/ecs/EcsCatalog.java | 3 ++- .../spark/extensions/SparkRowLevelOperationsTestBase.java | 2 +- .../spark/extensions/SparkRowLevelOperationsTestBase.java | 2 +- 8 files changed, 15 insertions(+), 11 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java index a957b3096a0b..fc1479c3a007 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java @@ -49,6 +49,7 @@ import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -130,7 +131,7 @@ public void initialize(String name, Map properties) { void initialize( String name, String path, AwsProperties properties, DynamoDbClient client, FileIO io) { Preconditions.checkArgument( - path != null && path.length() > 0, + !Strings.isNullOrEmpty(path), "Cannot initialize DynamoDbCatalog because warehousePath must not be null or empty"); this.catalogName = name; diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java index 0a758212088c..6e95379c1de5 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java @@ -55,6 +55,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -189,8 +190,7 @@ void initialize( this.catalogName = name; this.awsProperties = properties; this.s3FileIOProperties = s3Properties; - this.warehousePath = - (path != null && path.length() > 0) ? LocationUtil.stripTrailingSlash(path) : null; + this.warehousePath = Strings.isNullOrEmpty(path) ? null : LocationUtil.stripTrailingSlash(path); this.glue = client; this.lockManager = lock; @@ -286,7 +286,7 @@ protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) { } ValidationException.check( - warehousePath != null && warehousePath.length() > 0, + !Strings.isNullOrEmpty(warehousePath), "Cannot derive default warehouse location, warehouse path must not be null or empty"); return String.format( diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java index 25e6994d0bde..166551fe5575 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopCatalog.java @@ -54,6 +54,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -104,7 +105,7 @@ public void initialize(String name, Map properties) { this.catalogProperties = ImmutableMap.copyOf(properties); String inputWarehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); Preconditions.checkArgument( - inputWarehouseLocation != null && inputWarehouseLocation.length() > 0, + !Strings.isNullOrEmpty(inputWarehouseLocation), "Cannot initialize HadoopCatalog because warehousePath must not be null or empty"); this.catalogName = name; diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index c45df884bdfd..314595dd0255 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -53,6 +53,7 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -99,7 +100,7 @@ public void initialize(String name, Map properties) { String inputWarehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); Preconditions.checkArgument( - inputWarehouseLocation != null && inputWarehouseLocation.length() > 0, + !Strings.isNullOrEmpty(inputWarehouseLocation), "Cannot initialize JDBCCatalog because warehousePath must not be null or empty"); this.warehouseLocation = LocationUtil.stripTrailingSlash(inputWarehouseLocation); diff --git a/core/src/main/java/org/apache/iceberg/util/LocationUtil.java b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java index 8eca0051ccb9..192786155785 100644 --- a/core/src/main/java/org/apache/iceberg/util/LocationUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java @@ -19,13 +19,13 @@ package org.apache.iceberg.util; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; public class LocationUtil { private LocationUtil() {} public static String stripTrailingSlash(String path) { - Preconditions.checkArgument( - path != null && path.length() > 0, "path must not be null or empty"); + Preconditions.checkArgument(!Strings.isNullOrEmpty(path), "path must not be null or empty"); String result = path; while (result.endsWith("/")) { diff --git a/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsCatalog.java b/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsCatalog.java index bf16e1b07f76..f951c8c937ea 100644 --- a/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsCatalog.java +++ b/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsCatalog.java @@ -55,6 +55,7 @@ import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; @@ -99,7 +100,7 @@ public void initialize(String name, Map properties) { this.catalogProperties = ImmutableMap.copyOf(properties); String inputWarehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); Preconditions.checkArgument( - inputWarehouseLocation != null && inputWarehouseLocation.length() > 0, + !Strings.isNullOrEmpty(inputWarehouseLocation), "Cannot initialize EcsCatalog because warehousePath must not be null or empty"); this.catalogName = name; diff --git a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 633b2ee431ea..e53c79f6c971 100644 --- a/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -212,7 +212,7 @@ protected void createOrReplaceView(String name, List data, Encoder enc private Dataset toDS(String schema, String jsonData) { List jsonRows = Arrays.stream(jsonData.split("\n")) - .filter(str -> str.trim().length() > 0) + .filter(str -> !str.trim().isEmpty()) .collect(Collectors.toList()); Dataset jsonDS = spark.createDataset(jsonRows, Encoders.STRING()); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index ceb6b6aad87e..9a1257ba39bf 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -225,7 +225,7 @@ protected void createOrReplaceView(String name, List data, Encoder enc private Dataset toDS(String schema, String jsonData) { List jsonRows = Arrays.stream(jsonData.split("\n")) - .filter(str -> str.trim().length() > 0) + .filter(str -> !str.trim().isEmpty()) .collect(Collectors.toList()); Dataset jsonDS = spark.createDataset(jsonRows, Encoders.STRING()); From b21a8ce2497eb560feb9ed771a1e28ca57d1cf60 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Tue, 28 Nov 2023 10:41:38 -0800 Subject: [PATCH 69/94] API: add StructTransform base class for PartitionKey and SortKey. add SortOrderComparators (#7798) --- .../java/org/apache/iceberg/PartitionKey.java | 97 +--- .../main/java/org/apache/iceberg/SortKey.java | 56 ++ .../apache/iceberg/SortOrderComparators.java | 105 ++++ .../org/apache/iceberg/StructTransform.java | 149 ++++++ .../iceberg/TestSortOrderComparators.java | 489 ++++++++++++++++++ 5 files changed, 814 insertions(+), 82 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/SortKey.java create mode 100644 api/src/main/java/org/apache/iceberg/SortOrderComparators.java create mode 100644 api/src/main/java/org/apache/iceberg/StructTransform.java create mode 100644 api/src/test/java/org/apache/iceberg/TestSortOrderComparators.java diff --git a/api/src/main/java/org/apache/iceberg/PartitionKey.java b/api/src/main/java/org/apache/iceberg/PartitionKey.java index fc56d1a45347..08e3ea319b32 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionKey.java +++ b/api/src/main/java/org/apache/iceberg/PartitionKey.java @@ -18,13 +18,8 @@ */ package org.apache.iceberg; -import java.io.Serializable; -import java.lang.reflect.Array; -import java.util.Arrays; import java.util.List; -import java.util.function.Function; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.SerializableFunction; +import java.util.stream.Collectors; /** * A struct of partition values. @@ -32,58 +27,23 @@ *

Instances of this class can produce partition values from a data row passed to {@link * #partition(StructLike)}. */ -public class PartitionKey implements StructLike, Serializable { +public class PartitionKey extends StructTransform { private final PartitionSpec spec; - private final int size; - private final Object[] partitionTuple; - private final SerializableFunction[] transforms; - private final Accessor[] accessors; + private final Schema inputSchema; @SuppressWarnings("unchecked") public PartitionKey(PartitionSpec spec, Schema inputSchema) { + super(inputSchema, fieldTransform(spec)); this.spec = spec; - - List fields = spec.fields(); - this.size = fields.size(); - this.partitionTuple = new Object[size]; - this.transforms = new SerializableFunction[size]; - this.accessors = (Accessor[]) Array.newInstance(Accessor.class, size); - - Schema schema = spec.schema(); - for (int i = 0; i < size; i += 1) { - PartitionField field = fields.get(i); - Accessor accessor = inputSchema.accessorForField(field.sourceId()); - Preconditions.checkArgument( - accessor != null, - "Cannot build accessor for field: " + schema.findField(field.sourceId())); - this.accessors[i] = accessor; - this.transforms[i] = field.transform().bind(accessor.type()); - } + this.inputSchema = inputSchema; } private PartitionKey(PartitionKey toCopy) { + // only need deep copy inside StructTransform + super(toCopy); this.spec = toCopy.spec; - this.size = toCopy.size; - this.partitionTuple = new Object[toCopy.partitionTuple.length]; - this.transforms = toCopy.transforms; - this.accessors = toCopy.accessors; - - System.arraycopy(toCopy.partitionTuple, 0, this.partitionTuple, 0, partitionTuple.length); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append("["); - for (int i = 0; i < partitionTuple.length; i += 1) { - if (i > 0) { - sb.append(", "); - } - sb.append(partitionTuple[i]); - } - sb.append("]"); - return sb.toString(); + this.inputSchema = toCopy.inputSchema; } public PartitionKey copy() { @@ -101,41 +61,14 @@ public String toPath() { */ @SuppressWarnings("unchecked") public void partition(StructLike row) { - for (int i = 0; i < partitionTuple.length; i += 1) { - Function transform = transforms[i]; - partitionTuple[i] = transform.apply(accessors[i].get(row)); - } - } - - @Override - public int size() { - return size; - } - - @Override - public T get(int pos, Class javaClass) { - return javaClass.cast(partitionTuple[pos]); - } - - @Override - public void set(int pos, T value) { - partitionTuple[pos] = value; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } else if (!(o instanceof PartitionKey)) { - return false; - } - - PartitionKey that = (PartitionKey) o; - return Arrays.equals(partitionTuple, that.partitionTuple); + wrap(row); } - @Override - public int hashCode() { - return Arrays.hashCode(partitionTuple); + private static List fieldTransform(PartitionSpec spec) { + return spec.fields().stream() + .map( + partitionField -> + new FieldTransform(partitionField.sourceId(), partitionField.transform())) + .collect(Collectors.toList()); } } diff --git a/api/src/main/java/org/apache/iceberg/SortKey.java b/api/src/main/java/org/apache/iceberg/SortKey.java new file mode 100644 index 000000000000..8a9ccf0bd7c6 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/SortKey.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * A struct of flattened sort field values. + * + *

Instances of this class can produce sort values from a row passed to {@link + * #wrap(StructLike)}. + */ +public class SortKey extends StructTransform { + private final Schema schema; + private final SortOrder sortOrder; + + public SortKey(Schema schema, SortOrder sortOrder) { + super(schema, fieldTransform(sortOrder)); + this.schema = schema; + this.sortOrder = sortOrder; + } + + private SortKey(SortKey toCopy) { + // only need deep copy inside StructTransform + super(toCopy); + this.schema = toCopy.schema; + this.sortOrder = toCopy.sortOrder; + } + + public SortKey copy() { + return new SortKey(this); + } + + private static List fieldTransform(SortOrder sortOrder) { + return sortOrder.fields().stream() + .map(sortField -> new FieldTransform(sortField.sourceId(), sortField.transform())) + .collect(Collectors.toList()); + } +} diff --git a/api/src/main/java/org/apache/iceberg/SortOrderComparators.java b/api/src/main/java/org/apache/iceberg/SortOrderComparators.java new file mode 100644 index 000000000000..c730127c54e8 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/SortOrderComparators.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.lang.reflect.Array; +import java.util.Comparator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +public class SortOrderComparators { + private SortOrderComparators() {} + + /** Compare structs with the specified sort order projection */ + public static Comparator forSchema(Schema schema, SortOrder sortOrder) { + Preconditions.checkArgument(sortOrder.isSorted(), "Invalid sort order: unsorted"); + SortOrder.checkCompatibility(sortOrder, schema); + return new SortOrderComparator(schema, sortOrder); + } + + /** Util method to chain sort direction and null order to the original comparator. */ + private static Comparator sortFieldComparator( + Comparator original, SortField sortField) { + Comparator comparator = original; + if (sortField == null) { + return Comparators.nullsFirst().thenComparing(comparator); + } + + if (sortField.direction() == SortDirection.DESC) { + comparator = comparator.reversed(); + } + + if (sortField.nullOrder() == NullOrder.NULLS_FIRST) { + comparator = Comparators.nullsFirst().thenComparing(comparator); + } else if (sortField.nullOrder() == NullOrder.NULLS_LAST) { + comparator = Comparators.nullsLast().thenComparing(comparator); + } + + return comparator; + } + + private static class SortOrderComparator implements Comparator { + private final SortKey leftKey; + private final SortKey rightKey; + private final int size; + private final Comparator[] comparators; + private final Type[] transformResultTypes; + + private SortOrderComparator(Schema schema, SortOrder sortOrder) { + this.leftKey = new SortKey(schema, sortOrder); + this.rightKey = new SortKey(schema, sortOrder); + this.size = sortOrder.fields().size(); + this.comparators = (Comparator[]) Array.newInstance(Comparator.class, size); + this.transformResultTypes = (Type[]) Array.newInstance(Type.class, size); + + for (int i = 0; i < size; ++i) { + SortField sortField = sortOrder.fields().get(i); + Types.NestedField field = schema.findField(sortField.sourceId()); + Type transformResultType = sortField.transform().getResultType(field.type()); + Preconditions.checkArgument( + transformResultType.isPrimitiveType(), "Invalid transform result type: non-primitive"); + transformResultTypes[i] = transformResultType; + Comparator comparator = Comparators.forType(transformResultType.asPrimitiveType()); + comparators[i] = sortFieldComparator(comparator, sortField); + } + } + + @Override + public int compare(StructLike left, StructLike right) { + if (left == right) { + return 0; + } + + leftKey.wrap(left); + rightKey.wrap(right); + + for (int i = 0; i < size; i += 1) { + Class valueClass = transformResultTypes[i].typeId().javaClass(); + int cmp = comparators[i].compare(leftKey.get(i, valueClass), rightKey.get(i, valueClass)); + if (cmp != 0) { + return cmp; + } + } + + return 0; + } + } +} diff --git a/api/src/main/java/org/apache/iceberg/StructTransform.java b/api/src/main/java/org/apache/iceberg/StructTransform.java new file mode 100644 index 000000000000..dcb27488a5cc --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/StructTransform.java @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.io.Serializable; +import java.lang.reflect.Array; +import java.util.Arrays; +import java.util.List; +import java.util.function.Function; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.util.SerializableFunction; + +/** + * A struct of flattened transformed values. + * + *

Instances of this class can produce transformed values from a row passed to {@link + * #wrap(StructLike)}. + */ +class StructTransform implements StructLike, Serializable { + + private final int size; + private final Accessor[] accessors; + + @SuppressWarnings("rawtypes") + private final SerializableFunction[] transforms; + + private final Object[] transformedTuple; + + StructTransform(Schema schema, List fieldTransforms) { + Preconditions.checkArgument(fieldTransforms != null, "Invalid field transform list: null"); + + this.size = fieldTransforms.size(); + this.accessors = (Accessor[]) Array.newInstance(Accessor.class, size); + this.transforms = new SerializableFunction[size]; + + for (int i = 0; i < size; ++i) { + int sourceFieldId = fieldTransforms.get(i).sourceFieldId(); + Transform transform = fieldTransforms.get(i).transform(); + Accessor accessor = schema.accessorForField(sourceFieldId); + Preconditions.checkArgument( + accessor != null, "Cannot build accessor for field: %s", schema.findField(sourceFieldId)); + this.accessors[i] = accessor; + this.transforms[i] = transform.bind(accessor.type()); + } + + this.transformedTuple = new Object[size]; + } + + StructTransform(StructTransform toCopy) { + this.size = toCopy.size; + this.accessors = toCopy.accessors; + this.transforms = toCopy.transforms; + + this.transformedTuple = new Object[size]; + System.arraycopy(toCopy.transformedTuple, 0, this.transformedTuple, 0, size); + } + + public void wrap(StructLike row) { + for (int i = 0; i < transformedTuple.length; i += 1) { + Function transform = transforms[i]; + transformedTuple[i] = transform.apply(accessors[i].get(row)); + } + } + + @Override + public int size() { + return size; + } + + @Override + public T get(int pos, Class javaClass) { + return javaClass.cast(transformedTuple[pos]); + } + + @Override + public void set(int pos, T value) { + transformedTuple[pos] = value; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("["); + for (int i = 0; i < transformedTuple.length; i += 1) { + if (i > 0) { + sb.append(", "); + } + sb.append(transformedTuple[i]); + } + sb.append("]"); + return sb.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } else if (!(o instanceof StructTransform)) { + return false; + } + + StructTransform that = (StructTransform) o; + return Arrays.equals(transformedTuple, that.transformedTuple); + } + + @Override + public int hashCode() { + return Arrays.hashCode(transformedTuple); + } + + /** + * Simple POJO for source field id and transform function. {@code Pair} class is not usable here + * in API module, as it has an Avro dep and is in the core module. + */ + static class FieldTransform { + private final int sourceFieldId; + private final Transform transform; + + FieldTransform(int sourceFieldId, Transform transform) { + this.sourceFieldId = sourceFieldId; + this.transform = transform; + } + + int sourceFieldId() { + return sourceFieldId; + } + + Transform transform() { + return transform; + } + } +} diff --git a/api/src/test/java/org/apache/iceberg/TestSortOrderComparators.java b/api/src/test/java/org/apache/iceberg/TestSortOrderComparators.java new file mode 100644 index 000000000000..90ac3e09b6dd --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/TestSortOrderComparators.java @@ -0,0 +1,489 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.Comparator; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SerializableFunction; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestSortOrderComparators { + + private static void assertComparesCorrectly( + Schema schema, + SortOrder sortOrder, + StructLike less, + StructLike greater, + StructLike lessCopy, + StructLike nullValue) { + + Comparator comparator = SortOrderComparators.forSchema(schema, sortOrder); + // all fields should have the same sort direction in this test class + Assertions.assertThat(sortOrder.fields().stream().map(SortField::direction).distinct()) + .hasSize(1); + SortDirection direction = sortOrder.fields().get(0).direction(); + + Assertions.assertThat(comparator.compare(less, less)).isEqualTo(0); + Assertions.assertThat(comparator.compare(greater, greater)).isEqualTo(0); + Assertions.assertThat(comparator.compare(less, lessCopy)).isEqualTo(0); + + if (direction == SortDirection.ASC) { + Assertions.assertThat(comparator.compare(less, greater)).isEqualTo(-1); + Assertions.assertThat(comparator.compare(greater, less)).isEqualTo(1); + // null first + Assertions.assertThat(comparator.compare(nullValue, less)).isEqualTo(-1); + Assertions.assertThat(comparator.compare(less, nullValue)).isEqualTo(1); + } else { + Assertions.assertThat(comparator.compare(less, greater)).isEqualTo(1); + Assertions.assertThat(comparator.compare(greater, less)).isEqualTo(-1); + // null last + Assertions.assertThat(comparator.compare(nullValue, greater)).isEqualTo(1); + Assertions.assertThat(comparator.compare(less, nullValue)).isEqualTo(-1); + } + } + + private static Schema singleSortFildSchema(Type type) { + return new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "field", type)); + } + + private static final TestHelpers.Row SINGLE_SORT_FILED_NULL_VALUE = + TestHelpers.Row.of("id4", null); + + private static SortOrder ascOrder(Schema schema) { + return SortOrder.builderFor(schema).asc("field").build(); + } + + private static SortOrder descOrder(Schema schema) { + return SortOrder.builderFor(schema).desc("field", NullOrder.NULLS_LAST).build(); + } + + private static void testIdentitySortField( + Schema schema, TestHelpers.Row less, TestHelpers.Row greater, TestHelpers.Row lessCopy) { + assertComparesCorrectly( + schema, ascOrder(schema), less, greater, lessCopy, SINGLE_SORT_FILED_NULL_VALUE); + assertComparesCorrectly( + schema, descOrder(schema), less, greater, lessCopy, SINGLE_SORT_FILED_NULL_VALUE); + } + + @Test + public void testBoolean() { + testIdentitySortField( + singleSortFildSchema(Types.BooleanType.get()), + TestHelpers.Row.of("id3", Boolean.FALSE), + TestHelpers.Row.of("id2", Boolean.TRUE), + TestHelpers.Row.of("id1", Boolean.FALSE)); + } + + @Test + public void testInt() { + testIdentitySortField( + singleSortFildSchema(Types.IntegerType.get()), + TestHelpers.Row.of("id3", 111), + TestHelpers.Row.of("id2", 222), + TestHelpers.Row.of("id1", 111)); + } + + @Test + public void testLong() { + testIdentitySortField( + singleSortFildSchema(Types.LongType.get()), + TestHelpers.Row.of("id3", 111L), + TestHelpers.Row.of("id2", 222L), + TestHelpers.Row.of("id1", 111L)); + } + + @Test + public void testFloat() { + testIdentitySortField( + singleSortFildSchema(Types.FloatType.get()), + TestHelpers.Row.of("id3", 1.11f), + TestHelpers.Row.of("id1", 2.22f), + TestHelpers.Row.of("id1", 1.11f)); + } + + @Test + public void testDouble() { + testIdentitySortField( + singleSortFildSchema(Types.DoubleType.get()), + TestHelpers.Row.of("id3", 1.11d), + TestHelpers.Row.of("id2", 2.22d), + TestHelpers.Row.of("id1", 1.11d)); + } + + @Test + public void testDate() { + testIdentitySortField( + singleSortFildSchema(Types.DateType.get()), + TestHelpers.Row.of("id3", 111), + TestHelpers.Row.of("id2", 222), + TestHelpers.Row.of("id1", 111)); + } + + @Test + public void testTime() { + testIdentitySortField( + singleSortFildSchema(Types.TimeType.get()), + TestHelpers.Row.of("id3", 111L), + TestHelpers.Row.of("id2", 222L), + TestHelpers.Row.of("id1", 111L)); + } + + @Test + public void testTimestamp() { + Schema schemaWithZone = singleSortFildSchema(Types.TimestampType.withZone()); + Schema schemaWithoutZone = singleSortFildSchema(Types.TimestampType.withoutZone()); + + long lessMicro = + TimeUnit.SECONDS.toMicros( + LocalDateTime.of(2022, 1, 10, 0, 0, 0).toEpochSecond(ZoneOffset.UTC)); + long greaterMicro = + TimeUnit.SECONDS.toMicros( + LocalDateTime.of(2022, 1, 10, 1, 0, 0).toEpochSecond(ZoneOffset.UTC)); + long lessCopyMicro = + TimeUnit.SECONDS.toMicros( + LocalDateTime.of(2022, 1, 10, 0, 0, 0).toEpochSecond(ZoneOffset.UTC)); + + TestHelpers.Row less = TestHelpers.Row.of("id3", lessMicro); + TestHelpers.Row greater = TestHelpers.Row.of("id2", greaterMicro); + TestHelpers.Row lessCopy = TestHelpers.Row.of("id1", lessCopyMicro); + testIdentitySortField(schemaWithZone, less, greater, lessCopy); + testIdentitySortField(schemaWithoutZone, less, greater, lessCopy); + } + + @Test + public void testTimestampTransform() { + Schema schema = singleSortFildSchema(Types.TimestampType.withZone()); + Transform bucket = Transforms.day(); + SerializableFunction transform = bucket.bind(Types.TimestampType.withZone()); + + long lessMicro = + TimeUnit.SECONDS.toMicros( + LocalDateTime.of(2022, 1, 10, 0, 0, 0).toEpochSecond(ZoneOffset.UTC)); + long greaterMicro = + TimeUnit.SECONDS.toMicros( + LocalDateTime.of(2022, 1, 11, 0, 0, 0).toEpochSecond(ZoneOffset.UTC)); + // intentionally changed hour value from 0 to 1. days should equal + long lessCopyMicro = + TimeUnit.SECONDS.toMicros( + LocalDateTime.of(2022, 1, 10, 1, 0, 0).toEpochSecond(ZoneOffset.UTC)); + + Assertions.assertThat(transform.apply(lessMicro)).isLessThan(transform.apply(greaterMicro)); + Assertions.assertThat(transform.apply(lessMicro)).isEqualTo(transform.apply(lessCopyMicro)); + + TestHelpers.Row less = TestHelpers.Row.of("id3", lessMicro); + TestHelpers.Row greater = TestHelpers.Row.of("id2", greaterMicro); + TestHelpers.Row lessCopy = TestHelpers.Row.of("id1", lessCopyMicro); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .sortBy(Expressions.day("field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .build(); + assertComparesCorrectly( + schema, sortOrder, less, greater, lessCopy, SINGLE_SORT_FILED_NULL_VALUE); + } + + @Test + public void testString() { + testIdentitySortField( + singleSortFildSchema(Types.StringType.get()), + TestHelpers.Row.of("id3", "aaa"), + TestHelpers.Row.of("id2", "bbb"), + TestHelpers.Row.of("id1", "aaa")); + } + + @Test + public void testStringBucket() { + Schema schema = singleSortFildSchema(Types.StringType.get()); + Transform bucket = Transforms.bucket(4); + SerializableFunction transform = bucket.bind(Types.StringType.get()); + + Assertions.assertThat(transform.apply("bbb")).isLessThan(transform.apply("aaa")); + Assertions.assertThat(transform.apply("bbb")).isEqualTo(transform.apply("cca")); + + TestHelpers.Row less = TestHelpers.Row.of("id3", "bbb"); + TestHelpers.Row greater = TestHelpers.Row.of("id2", "aaa"); + // bucket("cca", 4) equals to bucket("bbb", 4) + TestHelpers.Row lessCopy = TestHelpers.Row.of("id1", "cca"); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .sortBy(Expressions.bucket("field", 4), SortDirection.ASC, NullOrder.NULLS_FIRST) + .build(); + assertComparesCorrectly( + schema, sortOrder, less, greater, lessCopy, SINGLE_SORT_FILED_NULL_VALUE); + } + + @Test + public void testUuid() { + Schema schema = singleSortFildSchema(Types.UUIDType.get()); + TestHelpers.Row less = + TestHelpers.Row.of("id3", UUID.fromString("81873e7d-1374-4493-8e1d-9095eff7046c")); + TestHelpers.Row greater = + TestHelpers.Row.of("id2", UUID.fromString("fd02441d-1423-4a3f-8785-c7dd5647e26b")); + TestHelpers.Row lessCopy = + TestHelpers.Row.of("id1", UUID.fromString("81873e7d-1374-4493-8e1d-9095eff7046c")); + testIdentitySortField(schema, less, greater, lessCopy); + } + + @Test + public void testUUIDBucket() { + Schema schema = singleSortFildSchema(Types.UUIDType.get()); + Transform bucket = Transforms.bucket(4); + SerializableFunction transform = bucket.bind(Types.UUIDType.get()); + + Assertions.assertThat(transform.apply(UUID.fromString("fd02441d-1423-4a3f-8785-c7dd5647e26b"))) + .isLessThan(transform.apply(UUID.fromString("86873e7d-1374-4493-8e1d-9095eff7046c"))); + Assertions.assertThat(transform.apply(UUID.fromString("fd02441d-1423-4a3f-8785-c7dd5647e26b"))) + .isEqualTo(transform.apply(UUID.fromString("81873e7d-1374-4493-8e1d-9095eff7046c"))); + + TestHelpers.Row less = + TestHelpers.Row.of("id3", UUID.fromString("fd02441d-1423-4a3f-8785-c7dd5647e26b")); + TestHelpers.Row greater = + TestHelpers.Row.of("id2", UUID.fromString("86873e7d-1374-4493-8e1d-9095eff7046c")); + // bucket value equals to the less object + TestHelpers.Row lessCopy = + TestHelpers.Row.of("id1", UUID.fromString("81873e7d-1374-4493-8e1d-9095eff7046c")); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .sortBy(Expressions.bucket("field", 4), SortDirection.ASC, NullOrder.NULLS_FIRST) + .build(); + assertComparesCorrectly( + schema, sortOrder, less, greater, lessCopy, SINGLE_SORT_FILED_NULL_VALUE); + } + + @Test + public void testFixed() { + testIdentitySortField( + singleSortFildSchema(Types.FixedType.ofLength(3)), + TestHelpers.Row.of("id3", ByteBuffer.wrap(new byte[] {1, 2, 3})), + TestHelpers.Row.of("id2", ByteBuffer.wrap(new byte[] {3, 2, 1})), + TestHelpers.Row.of("id1", ByteBuffer.wrap(new byte[] {1, 2, 3}))); + } + + @Test + public void testBinary() { + testIdentitySortField( + singleSortFildSchema(Types.BinaryType.get()), + TestHelpers.Row.of("id3", ByteBuffer.wrap(new byte[] {1, 1})), + TestHelpers.Row.of("id2", ByteBuffer.wrap(new byte[] {1, 1, 1})), + TestHelpers.Row.of("id1", ByteBuffer.wrap(new byte[] {1, 1}))); + } + + @Test + public void testBinaryTruncate() { + Schema schema = singleSortFildSchema(Types.BinaryType.get()); + Transform truncate = Transforms.truncate(2); + SerializableFunction transform = truncate.bind(Types.BinaryType.get()); + + Assertions.assertThat(transform.apply(ByteBuffer.wrap(new byte[] {1, 2, 3}))) + .isLessThan(transform.apply(ByteBuffer.wrap(new byte[] {1, 3, 1}))); + Assertions.assertThat(transform.apply(ByteBuffer.wrap(new byte[] {1, 2, 3}))) + .isEqualTo(transform.apply(ByteBuffer.wrap(new byte[] {1, 2, 5, 6}))); + + TestHelpers.Row less = TestHelpers.Row.of("id3", ByteBuffer.wrap(new byte[] {1, 2, 3})); + TestHelpers.Row greater = TestHelpers.Row.of("id2", ByteBuffer.wrap(new byte[] {1, 3, 1})); + // bucket value equals to the less object + TestHelpers.Row lessCopy = TestHelpers.Row.of("id1", ByteBuffer.wrap(new byte[] {1, 2, 5, 6})); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .sortBy(Expressions.truncate("field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .build(); + assertComparesCorrectly( + schema, sortOrder, less, greater, lessCopy, SINGLE_SORT_FILED_NULL_VALUE); + } + + @Test + public void testDecimal() { + testIdentitySortField( + singleSortFildSchema(Types.DecimalType.of(9, 5)), + TestHelpers.Row.of("id3", BigDecimal.valueOf(0.1)), + TestHelpers.Row.of("id2", BigDecimal.valueOf(0.2)), + TestHelpers.Row.of("id1", BigDecimal.valueOf(0.1))); + } + + @Test + public void testStruct() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.optional( + 2, + "location", + Types.StructType.of( + Types.NestedField.required(11, "lat", Types.FloatType.get()), + Types.NestedField.required(12, "long", Types.FloatType.get())))); + + SortOrder latOnlyAsc = SortOrder.builderFor(schema).asc("location.lat").build(); + TestHelpers.Row lessLat = TestHelpers.Row.of("id4", TestHelpers.Row.of(1.0f, 1.0f)); + TestHelpers.Row greaterLat = TestHelpers.Row.of("id3", TestHelpers.Row.of(2.0f, 1.0f)); + TestHelpers.Row lessLatCopy = TestHelpers.Row.of("id2", TestHelpers.Row.of(1.0f, 1.0f)); + TestHelpers.Row nullLat = TestHelpers.Row.of("id1", TestHelpers.Row.of(null, 1.0f)); + assertComparesCorrectly(schema, latOnlyAsc, lessLat, greaterLat, lessLatCopy, nullLat); + + SortOrder longOnlyDesc = + SortOrder.builderFor(schema).desc("location.long", NullOrder.NULLS_LAST).build(); + TestHelpers.Row lessLong = TestHelpers.Row.of("id4", TestHelpers.Row.of(1.0f, 1.0f)); + TestHelpers.Row greaterLong = TestHelpers.Row.of("id3", TestHelpers.Row.of(1.0f, 2.0f)); + TestHelpers.Row lessLongCopy = TestHelpers.Row.of("id2", TestHelpers.Row.of(1.0f, 1.0f)); + TestHelpers.Row nullLong = TestHelpers.Row.of("id1", TestHelpers.Row.of(1.0f, null)); + assertComparesCorrectly(schema, longOnlyDesc, lessLong, greaterLong, lessLongCopy, nullLong); + + SortOrder twoFieldsSortOrder = + SortOrder.builderFor(schema).asc("location.lat").asc("location.long").build(); + TestHelpers.Row lessTwoSortFields = TestHelpers.Row.of("id4", TestHelpers.Row.of(1.0f, 1.0f)); + TestHelpers.Row greaterLatTwoSortFields = + TestHelpers.Row.of("id3", TestHelpers.Row.of(2.0f, 1.0f)); + TestHelpers.Row greaterLongTwoSortFields = + TestHelpers.Row.of("id3", TestHelpers.Row.of(1.0f, 2.0f)); + TestHelpers.Row greaterBothTwoSortFields = + TestHelpers.Row.of("id3", TestHelpers.Row.of(2.0f, 2.0f)); + TestHelpers.Row lessTwoSortFieldsCopy = + TestHelpers.Row.of("id2", TestHelpers.Row.of(1.0f, 1.0f)); + TestHelpers.Row nullLatTwoSortFields = + TestHelpers.Row.of("id1", TestHelpers.Row.of(null, 1.0f)); + TestHelpers.Row nullLongTwoSortFields = + TestHelpers.Row.of("id1", TestHelpers.Row.of(1.0f, null)); + TestHelpers.Row nullBothTowSortFields = + TestHelpers.Row.of("id1", TestHelpers.Row.of(null, null)); + assertComparesCorrectly( + schema, + twoFieldsSortOrder, + lessTwoSortFields, + greaterLatTwoSortFields, + lessTwoSortFieldsCopy, + nullLatTwoSortFields); + assertComparesCorrectly( + schema, + twoFieldsSortOrder, + lessTwoSortFields, + greaterLongTwoSortFields, + lessTwoSortFieldsCopy, + nullLongTwoSortFields); + assertComparesCorrectly( + schema, + twoFieldsSortOrder, + lessTwoSortFields, + greaterBothTwoSortFields, + lessTwoSortFieldsCopy, + nullBothTowSortFields); + } + + @Test + public void testStructTransform() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.optional( + 2, + "struct", + Types.StructType.of( + Types.NestedField.required(11, "left", Types.BinaryType.get()), + Types.NestedField.required(12, "right", Types.BinaryType.get())))); + + Transform bucket = Transforms.truncate(2); + SerializableFunction transform = bucket.bind(Types.BinaryType.get()); + + Assertions.assertThat(transform.apply(ByteBuffer.wrap(new byte[] {2, 3, 4}))) + .isLessThan(transform.apply(ByteBuffer.wrap(new byte[] {9, 3, 4}))); + Assertions.assertThat(transform.apply(ByteBuffer.wrap(new byte[] {2, 3, 4}))) + .isEqualTo(transform.apply(ByteBuffer.wrap(new byte[] {2, 3, 9}))); + + TestHelpers.Row less = + TestHelpers.Row.of( + "id4", + TestHelpers.Row.of( + ByteBuffer.wrap(new byte[] {1}), ByteBuffer.wrap(new byte[] {2, 3, 4}))); + TestHelpers.Row greater = + TestHelpers.Row.of( + "id3", + TestHelpers.Row.of( + ByteBuffer.wrap(new byte[] {1}), ByteBuffer.wrap(new byte[] {9, 3, 4}))); + TestHelpers.Row lessCopy = + TestHelpers.Row.of( + "id2", + TestHelpers.Row.of( + ByteBuffer.wrap(new byte[] {1}), ByteBuffer.wrap(new byte[] {2, 3, 9}))); + TestHelpers.Row nullRight = + TestHelpers.Row.of("id1", TestHelpers.Row.of(ByteBuffer.wrap(new byte[] {1}), null)); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .sortBy( + Expressions.truncate("struct.left", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("struct.right", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .build(); + assertComparesCorrectly(schema, sortOrder, less, greater, lessCopy, nullRight); + } + + @Test + public void testNestedStruct() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.optional( + 2, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.optional( + 12, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get())))))); + + SortOrder build = + SortOrder.builderFor(schema).asc("user.location.lat").asc("user.location.long").build(); + TestHelpers.Row less = + TestHelpers.Row.of("id4", TestHelpers.Row.of("user4", TestHelpers.Row.of(1.0f, 1.0f))); + TestHelpers.Row greaterLat = + TestHelpers.Row.of("id3", TestHelpers.Row.of("user3", TestHelpers.Row.of(2.0f, 1.0f))); + TestHelpers.Row greaterLong = + TestHelpers.Row.of("id3", TestHelpers.Row.of("user3", TestHelpers.Row.of(1.0f, 2.0f))); + TestHelpers.Row greaterBoth = + TestHelpers.Row.of("id3", TestHelpers.Row.of("user3", TestHelpers.Row.of(2.0f, 2.0f))); + TestHelpers.Row lessCopy = + TestHelpers.Row.of("id2", TestHelpers.Row.of("user2", TestHelpers.Row.of(1.0f, 1.0f))); + TestHelpers.Row nullLat = + TestHelpers.Row.of("id1", TestHelpers.Row.of("user1", TestHelpers.Row.of(null, 1.0f))); + TestHelpers.Row nullLong = + TestHelpers.Row.of("id1", TestHelpers.Row.of("user1", TestHelpers.Row.of(1.0f, null))); + TestHelpers.Row nullBoth = + TestHelpers.Row.of("id1", TestHelpers.Row.of("user1", TestHelpers.Row.of(null, null))); + assertComparesCorrectly(schema, build, less, greaterLat, lessCopy, nullLat); + assertComparesCorrectly(schema, build, less, greaterLong, lessCopy, nullLong); + assertComparesCorrectly(schema, build, less, greaterBoth, lessCopy, nullBoth); + } +} From 5fb26f8394b8b0b286aa0c7bb49426d6a21282e8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 28 Nov 2023 22:36:43 +0100 Subject: [PATCH 70/94] Build: Bump org.testcontainers:testcontainers from 1.19.2 to 1.19.3 (#9155) Bumps [org.testcontainers:testcontainers](https://github.com/testcontainers/testcontainers-java) from 1.19.2 to 1.19.3. - [Release notes](https://github.com/testcontainers/testcontainers-java/releases) - [Changelog](https://github.com/testcontainers/testcontainers-java/blob/main/CHANGELOG.md) - [Commits](https://github.com/testcontainers/testcontainers-java/compare/1.19.2...1.19.3) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 3b27bad336c8..1ee887d34520 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -83,7 +83,7 @@ spark-hive35 = "3.5.0" spring-boot = "2.5.4" spring-web = "5.3.30" sqlite-jdbc = "3.44.0.0" -testcontainers = "1.19.2" +testcontainers = "1.19.3" tez010 = "0.10.2" tez08 = { strictly = "[0.8, 0.9[", prefer = "0.8.4"} # see rich version usage explanation above From 5e059c1bfb664cc1880883425bfd68b2a8df3190 Mon Sep 17 00:00:00 2001 From: pvary Date: Tue, 28 Nov 2023 22:59:44 +0100 Subject: [PATCH 71/94] Flink: Backport #8553 to v1.15, v1.16 (#9145) --- .../iceberg/flink/source/IcebergSource.java | 68 ++- .../reader/ColumnStatsWatermarkExtractor.java | 98 ++++ .../source/reader/IcebergSourceReader.java | 3 +- .../reader/SerializableRecordEmitter.java | 40 ++ ...tter.java => SplitWatermarkExtractor.java} | 18 +- .../WatermarkExtractorRecordEmitter.java | 67 +++ .../flink/source/split/SplitComparators.java | 19 +- .../source/TestIcebergSourceFailover.java | 12 +- ...gSourceFailoverWithWatermarkExtractor.java | 112 ++++ ...stIcebergSourceWithWatermarkExtractor.java | 481 ++++++++++++++++++ .../assigner/SplitAssignerTestBase.java | 21 +- ...tFileSequenceNumberBasedSplitAssigner.java | 10 +- .../TestWatermarkBasedSplitAssigner.java | 146 ++++++ .../flink/source/reader/ReaderUtil.java | 8 +- .../TestColumnStatsWatermarkExtractor.java | 178 +++++++ .../reader/TestIcebergSourceReader.java | 7 +- .../iceberg/flink/source/IcebergSource.java | 68 ++- .../reader/ColumnStatsWatermarkExtractor.java | 98 ++++ .../source/reader/IcebergSourceReader.java | 3 +- .../reader/SerializableRecordEmitter.java | 40 ++ ...tter.java => SplitWatermarkExtractor.java} | 18 +- .../WatermarkExtractorRecordEmitter.java | 67 +++ .../flink/source/split/SplitComparators.java | 19 +- .../source/TestIcebergSourceFailover.java | 12 +- ...gSourceFailoverWithWatermarkExtractor.java | 112 ++++ ...stIcebergSourceWithWatermarkExtractor.java | 451 ++++++++++++++++ .../assigner/SplitAssignerTestBase.java | 21 +- ...tFileSequenceNumberBasedSplitAssigner.java | 10 +- .../TestWatermarkBasedSplitAssigner.java | 146 ++++++ .../flink/source/reader/ReaderUtil.java | 8 +- .../TestColumnStatsWatermarkExtractor.java | 178 +++++++ .../reader/TestIcebergSourceReader.java | 7 +- 32 files changed, 2460 insertions(+), 86 deletions(-) create mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java rename flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/{IcebergSourceRecordEmitter.java => SplitWatermarkExtractor.java} (63%) create mode 100644 flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java create mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java create mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java create mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java create mode 100644 flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java rename flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/{IcebergSourceRecordEmitter.java => SplitWatermarkExtractor.java} (63%) create mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index f85f6277263b..179253cb3a18 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.flink.annotation.Experimental; import org.apache.flink.api.connector.source.Boundedness; @@ -58,15 +59,20 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; +import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,6 +86,7 @@ public class IcebergSource implements Source readerFunction; private final SplitAssignerFactory assignerFactory; private final SerializableComparator splitComparator; + private final SerializableRecordEmitter emitter; // Can't use SerializableTable as enumerator needs a regular table // that can discover table changes @@ -91,13 +98,15 @@ public class IcebergSource implements Source readerFunction, SplitAssignerFactory assignerFactory, SerializableComparator splitComparator, - Table table) { + Table table, + SerializableRecordEmitter emitter) { this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; this.splitComparator = splitComparator; this.table = table; + this.emitter = emitter; } String name() { @@ -152,7 +161,8 @@ public Boundedness getBoundedness() { public SourceReader createReader(SourceReaderContext readerContext) { IcebergSourceReaderMetrics metrics = new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); - return new IcebergSourceReader<>(metrics, readerFunction, splitComparator, readerContext); + return new IcebergSourceReader<>( + emitter, metrics, readerFunction, splitComparator, readerContext); } @Override @@ -216,6 +226,8 @@ public static class Builder { private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; + private String watermarkColumn; + private TimeUnit watermarkTimeUnit = TimeUnit.MICROSECONDS; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -237,6 +249,9 @@ public Builder table(Table newTable) { } public Builder assignerFactory(SplitAssignerFactory assignerFactory) { + Preconditions.checkArgument( + watermarkColumn == null, + "Watermark column and SplitAssigner should not be set in the same source"); this.splitAssignerFactory = assignerFactory; return this; } @@ -429,6 +444,33 @@ public Builder setAll(Map properties) { return this; } + /** + * Emits watermarks once per split based on the min value of column statistics from files + * metadata in the given split. The generated watermarks are also used for ordering the splits + * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider + * setting {@link #watermarkTimeUnit(TimeUnit)}. + * + *

Consider setting `read.split.open-file-cost` to prevent combining small files to a single + * split when the watermark is used for watermark alignment. + */ + public Builder watermarkColumn(String columnName) { + Preconditions.checkArgument( + splitAssignerFactory == null, + "Watermark column and SplitAssigner should not be set in the same source"); + this.watermarkColumn = columnName; + return this; + } + + /** + * When the type of the {@link #watermarkColumn} is {@link + * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the + * value. The default value is {@link TimeUnit#MICROSECONDS}. + */ + public Builder watermarkTimeUnit(TimeUnit timeUnit) { + this.watermarkTimeUnit = timeUnit; + return this; + } + /** @deprecated Use {@link #setAll} instead. */ @Deprecated public Builder properties(Map properties) { @@ -453,6 +495,18 @@ public IcebergSource build() { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); } + SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); + if (watermarkColumn != null) { + // Column statistics is needed for watermark generation + contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); + + SplitWatermarkExtractor watermarkExtractor = + new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); + emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); + splitAssignerFactory = + new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); + } + ScanContext context = contextBuilder.build(); if (readerFunction == null) { if (table instanceof BaseMetadataTable) { @@ -485,8 +539,14 @@ public IcebergSource build() { checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading - return new IcebergSource( - tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table); + return new IcebergSource<>( + tableLoader, + context, + readerFunction, + splitAssignerFactory, + splitComparator, + table, + emitter); } private void checkRequired() { diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..4bb6f0a98c4c --- /dev/null +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; + +/** + * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics + * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link + * WatermarkExtractorRecordEmitter} along with the actual records. + */ +@Internal +public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { + private final int eventTimeFieldId; + private final String eventTimeFieldName; + private final TimeUnit timeUnit; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param eventTimeFieldName The column which should be used as an event time + * @param timeUnit Used for converting the long value to epoch milliseconds + */ + public ColumnStatsWatermarkExtractor( + Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { + Types.NestedField field = schema.findField(eventTimeFieldName); + TypeID typeID = field.type().typeId(); + Preconditions.checkArgument( + typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), + "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", + typeID); + this.eventTimeFieldId = field.fieldId(); + this.eventTimeFieldName = eventTimeFieldName; + // Use the timeUnit only for Long columns. + this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; + } + + @VisibleForTesting + ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { + this.eventTimeFieldId = eventTimeFieldId; + this.eventTimeFieldName = eventTimeFieldName; + this.timeUnit = TimeUnit.MICROSECONDS; + } + + /** + * Get the watermark for a split using column statistics. + * + * @param split The split + * @return The watermark + * @throws IllegalArgumentException if there is no statistics for the column + */ + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> { + Preconditions.checkArgument( + scanTask.file().lowerBounds() != null + && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, + "Missing statistics for column name = %s in file = %s", + eventTimeFieldName, + eventTimeFieldId, + scanTask.file()); + return timeUnit.toMillis( + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); + }) + .min(Comparator.comparingLong(l -> l)) + .get(); + } +} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java index 8d7d68f961cb..f143b8d2df2e 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java @@ -35,13 +35,14 @@ public class IcebergSourceReader RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { public IcebergSourceReader( + SerializableRecordEmitter emitter, IcebergSourceReaderMetrics metrics, ReaderFunction readerFunction, SerializableComparator splitComparator, SourceReaderContext context) { super( () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), - new IcebergSourceRecordEmitter<>(), + emitter, context.getConfiguration(), context); } diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java new file mode 100644 index 000000000000..a6e2c1dae243 --- /dev/null +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +@Internal +@FunctionalInterface +public interface SerializableRecordEmitter + extends RecordEmitter, T, IcebergSourceSplit>, Serializable { + static SerializableRecordEmitter defaultEmitter() { + return (element, output, split) -> { + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + }; + } + + static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { + return new WatermarkExtractorRecordEmitter<>(extractor); + } +} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 63% rename from flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java rename to flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java index 337d9d3c4223..d1c50ac8ca52 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java @@ -18,19 +18,11 @@ */ package org.apache.iceberg.flink.source.reader; -import org.apache.flink.api.connector.source.SourceOutput; -import org.apache.flink.connector.base.source.reader.RecordEmitter; +import java.io.Serializable; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -final class IcebergSourceRecordEmitter - implements RecordEmitter, T, IcebergSourceSplit> { - - IcebergSourceRecordEmitter() {} - - @Override - public void emitRecord( - RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - } +/** The interface used to extract watermarks from splits. */ +public interface SplitWatermarkExtractor extends Serializable { + /** Get the watermark for a split. */ + long extractWatermark(IcebergSourceSplit split); } diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java new file mode 100644 index 000000000000..02ef57d344b1 --- /dev/null +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the watermarks, records and updates the split position. + * + *

The Emitter emits watermarks at the beginning of every split provided by the {@link + * SplitWatermarkExtractor}. + */ +class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final SplitWatermarkExtractor timeExtractor; + private String lastSplitId = null; + private long watermark; + + WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { + this.timeExtractor = timeExtractor; + } + + @Override + public void emitRecord( + RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { + if (!split.splitId().equals(lastSplitId)) { + long newWatermark = timeExtractor.extractWatermark(split); + if (newWatermark < watermark) { + LOG.info( + "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", + watermark, + newWatermark, + lastSplitId, + split.splitId()); + } else { + watermark = newWatermark; + output.emitWatermark(new Watermark(watermark)); + LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); + } + + lastSplitId = split.splitId(); + } + + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + } +} diff --git a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index 64e03d77debe..56ee92014d12 100644 --- a/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.15/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.split; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** @@ -45,7 +46,7 @@ public static SerializableComparator fileSequenceNumber() { o1); Preconditions.checkNotNull( seq2, - "IInvalid file sequence number: null. Doesn't support splits written with V1 format: %s", + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", o2); int temp = Long.compare(seq1, seq2); @@ -56,4 +57,20 @@ public static SerializableComparator fileSequenceNumber() { } }; } + + /** Comparator which orders the splits based on watermark of the splits */ + public static SerializableComparator watermark( + SplitWatermarkExtractor watermarkExtractor) { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + long watermark1 = watermarkExtractor.extractWatermark(o1); + long watermark2 = watermarkExtractor.extractWatermark(o2); + + int temp = Long.compare(watermark1, watermark2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 70e7a79d8373..7d991ee603c9 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -39,6 +39,7 @@ import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -97,6 +98,11 @@ protected List generateRecords(int numRecords, long seed) { return RandomGenericData.generate(schema(), numRecords, seed); } + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); + } + @Test public void testBoundedWithTaskManagerFailover() throws Exception { testBoundedIcebergSource(FailoverType.TM); @@ -150,8 +156,7 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio RecordCounterToFail::continueProcessing, miniClusterResource.getMiniCluster()); - SimpleDataUtil.assertTableRecords( - sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } @Test @@ -214,8 +219,7 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - SimpleDataUtil.assertTableRecords( - sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } // ------------------------------------------------------------------------ diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java new file mode 100644 index 000000000000..f7dc931c506c --- /dev/null +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; + +public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { + // Increment ts by 15 minutes for each generateRecords batch + private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); + // Within a batch, increment ts by 1 second + private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); + + private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + + @Override + protected IcebergSource.Builder sourceBuilder() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA); + } + + @Override + protected Schema schema() { + return TestFixtures.TS_SCHEMA; + } + + @Override + protected List generateRecords(int numRecords, long seed) { + // Override the ts field to create a more realistic situation for event time alignment + tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); + return RandomGenericData.generate(schema(), numRecords, seed).stream() + .peek( + record -> { + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), + ZoneId.of("Z")); + record.setField("ts", ts); + }) + .collect(Collectors.toList()); + } + + /** + * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves + * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates + * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the + * {@link LocalDateTime} to a Long type so that Comparators can continue to work. + */ + @Override + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> { + SimpleDataUtil.equalsRecords( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + }); + } + + private List convertLocalDateTimeToMilli(List records) { + return records.stream() + .peek( + r -> { + LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); + r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java new file mode 100644 index 000000000000..7547323871c1 --- /dev/null +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -0,0 +1,481 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.operators.collect.CollectResultIterator; +import org.apache.flink.streaming.api.operators.collect.CollectSinkOperator; +import org.apache.flink.streaming.api.operators.collect.CollectSinkOperatorFactory; +import org.apache.flink.streaming.api.operators.collect.CollectStreamSink; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + private static final ConcurrentMap windows = Maps.newConcurrentMap(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + *

    + *
  • - Ordering of the splits + *
  • - Emitting of watermarks + *
  • - Firing windows based on watermarks + *
+ * + *

The test generates 4 splits + * + *

    + *
  • - Split 1 - Watermark 100 min + *
  • - Split 2, 3 - Watermark 0 min + *
  • - Split 4 - Watermark 6 min + *
+ * + *

Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + *

Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + *

    + *
  • - Split 2, 3 + *
  • - Split 4 + *
  • - Split 1 + *
+ * + *

As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + *

Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction() { + @Override + public void apply( + TimeWindow window, Iterable values, Collector out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicInteger count = new AtomicInteger(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + windows.put(window.getStart(), count.get()); + } + }); + + // Use static variable to collect the windows, since other solutions were flaky + windows.clear(); + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of( + 0L, + RECORD_NUM_FOR_2_SPLITS, + TimeUnit.MINUTES.toMillis(5), + 2, + TimeUnit.MINUTES.toMillis(100), + 3))); + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + *

    + *
  • - Emitting of watermarks + *
  • - Watermark alignment + *
+ * + *

The test generates 3 splits + * + *

    + *
  • - Split 1 - Watermark 100 min + *
  • - Split 2, 3 - Watermark 0 min + *
+ * + * The splits are read in the following order: + * + *
    + *
  • - Split 2, 3 (Task Manager 1, Task Manager 2) + *
  • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + *
+ * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + *

The status of the watermark alignment is checked by the alignment related metrics. + * + *

Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + *

After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + // Flink 1.15 only change - start + CollectResultIterator resultStream = addCollectSink(stream); + + // Start the job + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); + resultStream.setJobClient(jobClient); + try (CollectResultIterator resultIterator = resultStream) { + // Flink 1.15 only change - end + + // Check that the read the non-blocked data + // The first RECORD_NUM_FOR_2_SPLITS should be read + // 1 or more from the runaway reader should be arrived depending on thread scheduling + waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + List newBatch1 = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + List newBatch2 = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + dataAppender.appendToTable( + dataAppender.writeFile(newBatch1), dataAppender.writeFile(newBatch2)); + // The records received will highly depend on scheduling + // We minimally get 3 records from the non-blocked reader + // We might get 1 record from the blocked reader (as part of the previous batch - + // file_1-recordTs_100) + // We might get 3 records form the non-blocked reader if it gets both new splits + waitForRecords(resultIterator, 3); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); + + // Add some new records which should unblock the throttled reader + batch = + ImmutableList.of( + generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); + dataAppender.appendToTable(batch); + // We should get all the records at this point + waitForRecords(resultIterator, 6); + + // Wait for the new drift to decrease below the allowed drift to signal the normal state + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); + } + } + + protected IcebergSource source() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA) + .splitSize(100L) + .streaming(true) + .monitorInterval(Duration.ofMillis(2)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + } + + protected Record generateRecord(int minutes, String str) { + // Override the ts field to create a more realistic situation for event time alignment + Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), + ZoneId.of("Z")); + record.setField("ts", ts); + record.setField("str", str); + return record; + } + + protected void assertRecords( + Collection expectedRecords, CloseableIterator iterator) throws Exception { + Set expected = + expectedRecords.stream() + .map(e -> RowDataConverter.convert(TestFixtures.TS_SCHEMA, e)) + .collect(Collectors.toSet()); + Assert.assertEquals(expected, waitForRecords(iterator, expectedRecords.size())); + } + + protected Set waitForRecords(CloseableIterator iterator, int num) { + Set received = Sets.newHashSetWithExpectedSize(num); + assertThat( + CompletableFuture.supplyAsync( + () -> { + int count = 0; + while (count < num && iterator.hasNext()) { + received.add(iterator.next()); + count++; + } + + if (count < num) { + throw new IllegalStateException(String.format("Fail to get %d records.", num)); + } + + return true; + })) + .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); + + return received; + } + + private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { + String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; + return reporter.findMetrics(jobID, metricsName).values().stream() + .map(m -> (Gauge) m) + .filter(m -> m.getValue() == withValue) + .findFirst(); + } + + private GenericAppenderHelper appender() { + // We need to create multiple splits, so we need to generate parquet files with multiple offsets + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("write.parquet.page-size-bytes", "64"); + hadoopConf.set("write.parquet.row-group-size-bytes", "64"); + return new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + } + + private static RowData row(long time, long count) { + GenericRowData result = new GenericRowData(2); + result.setField(0, time); + result.setField(1, String.valueOf(count)); + return result; + } + + private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { + @Override + public long extractTimestamp(RowData element, long recordTimestamp) { + return element.getTimestamp(0, 0).getMillisecond(); + } + } + + // Flink 1.15 only method + private CollectResultIterator addCollectSink(DataStream stream) { + TypeSerializer serializer = + stream.getType().createSerializer(stream.getExecutionConfig()); + String accumulatorName = "dataStreamCollect_" + UUID.randomUUID(); + CollectSinkOperatorFactory factory = + new CollectSinkOperatorFactory<>(serializer, accumulatorName); + CollectSinkOperator operator = (CollectSinkOperator) factory.getOperator(); + CollectStreamSink sink = new CollectStreamSink<>(stream, factory); + sink.name("Data stream collect sink"); + stream.getExecutionEnvironment().addOperator(sink.getTransformation()); + return new CollectResultIterator<>( + operator.getOperatorIdFuture(), + serializer, + accumulatorName, + stream.getExecutionEnvironment().getCheckpointConfig()); + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index f28677ca9d6a..090b304942c6 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -43,15 +43,13 @@ public void testEmptyInitialization() { @Test public void testStaticEnumeratorSequence() throws Exception { SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 1)); + assigner.onDiscoveredSplits(createSplits(4, 1, "1")); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); - assigner.onUnassignedSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1)); + assigner.onUnassignedSplits(createSplits(1, 1, "1")); assertSnapshot(assigner, 2); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -66,15 +64,12 @@ public void testContinuousEnumeratorSequence() throws Exception { SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - List splits1 = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + List splits1 = createSplits(1, 1, "1"); assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); - List splits2 = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + List splits2 = createSplits(1, 1, "1"); assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 2, 1)); + assigner.onDiscoveredSplits(createSplits(2, 1, "1")); assertSnapshot(assigner, 2); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -125,5 +120,11 @@ protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Assert.assertEquals(splitCount, stateBeforeGet.size()); } + protected List createSplits(int fileCount, int filesPerSplit, String version) + throws Exception { + return SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, fileCount, filesPerSplit, version); + } + protected abstract SplitAssigner splitAssigner(); } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java index 8b9e132e0e22..e78634e6b873 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.iceberg.ContentFile; -import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; @@ -40,9 +39,7 @@ protected SplitAssigner splitAssigner() { public void testMultipleFilesInAnIcebergSplit() { SplitAssigner assigner = splitAssigner(); Assertions.assertThatThrownBy( - () -> - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2, "2")), + () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Please use 'split-open-file-cost'"); @@ -52,8 +49,7 @@ public void testMultipleFilesInAnIcebergSplit() { @Test public void testSplitSort() throws Exception { SplitAssigner assigner = splitAssigner(); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 5, 1, "2"); + List splits = createSplits(5, 1, "2"); assigner.onDiscoveredSplits(splits.subList(3, 5)); assigner.onDiscoveredSplits(splits.subList(0, 1)); @@ -76,7 +72,7 @@ public void testSerializable() { Assert.assertNotNull(comparator); } - protected void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { GetSplitResult result = assigner.getNext(null); ContentFile file = result.split().task().files().iterator().next().file(); Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java new file mode 100644 index 000000000000..e1fc63fda918 --- /dev/null +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SerializationUtil; +import org.junit.Assert; +import org.junit.Test; + +public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { + public static final Schema SCHEMA = + new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) + .createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 2, "2")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() { + SplitAssigner assigner = splitAssigner(); + + Instant now = Instant.now(); + List splits = + IntStream.range(0, 5) + .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) + .collect(Collectors.toList()); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, splits.get(0)); + assertGetNext(assigner, splits.get(1)); + assertGetNext(assigner, splits.get(2)); + assertGetNext(assigner, splits.get(3)); + assertGetNext(assigner, splits.get(4)); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = + SerializationUtil.serializeToBytes( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor( + TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { + GetSplitResult result = assigner.getNext(null); + Assert.assertEquals(result.split(), split); + } + + @Override + protected List createSplits( + int fileCount, int filesPerSplit, String version) { + return IntStream.range(0, fileCount / filesPerSplit) + .mapToObj( + splitNum -> + splitFromRecords( + IntStream.range(0, filesPerSplit) + .mapToObj( + fileNum -> + RandomGenericData.generate( + SCHEMA, 2, splitNum * filesPerSplit + fileNum)) + .collect(Collectors.toList()))) + .collect(Collectors.toList()); + } + + private IcebergSourceSplit splitFromInstant(Instant instant) { + Record record = GenericRecord.create(SCHEMA); + record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); + return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); + } + + private IcebergSourceSplit splitFromRecords(List> records) { + try { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + } catch (IOException e) { + throw new RuntimeException("Split creation exception", e); + } + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index f9ceaf842263..2a2503ef2478 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -60,9 +60,12 @@ public static FileScanTask createFileTask( FileFormat fileFormat, FileAppenderFactory appenderFactory) throws IOException { - try (FileAppender appender = - appenderFactory.newAppender(Files.localOutput(file), fileFormat)) { + FileAppender appender = + appenderFactory.newAppender(Files.localOutput(file), fileFormat); + try { appender.addAll(records); + } finally { + appender.close(); } DataFile dataFile = @@ -71,6 +74,7 @@ public static FileScanTask createFileTask( .withFileSizeInBytes(file.length()) .withPath(file.toString()) .withFormat(fileFormat) + .withMetrics(appender.metrics()) .build(); ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..afe8a5d0152c --- /dev/null +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestColumnStatsWatermarkExtractor { + public static final Schema SCHEMA = + new Schema( + required(1, "timestamp_column", Types.TimestampType.withoutZone()), + required(2, "timestamptz_column", Types.TimestampType.withZone()), + required(3, "long_column", Types.LongType.get()), + required(4, "string_column", Types.StringType.get())); + + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + private static final List> TEST_RECORDS = + ImmutableList.of( + RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); + + private static final List> MIN_VALUES = + ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); + + private final String columnName; + + @BeforeClass + public static void updateMinValue() { + for (int i = 0; i < TEST_RECORDS.size(); ++i) { + for (Record r : TEST_RECORDS.get(i)) { + Map minValues = MIN_VALUES.get(i); + + LocalDateTime localDateTime = (LocalDateTime) r.get(0); + minValues.merge( + "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); + + OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); + minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); + + minValues.merge("long_column", (Long) r.get(2), Math::min); + } + } + } + + @Parameterized.Parameters(name = "{0}") + public static Collection data() { + return ImmutableList.of( + new Object[] {"timestamp_column"}, + new Object[] {"timestamptz_column"}, + new Object[] {"long_column"}); + } + + public TestColumnStatsWatermarkExtractor(String columnName) { + this.columnName = columnName; + } + + @Test + public void testSingle() throws IOException { + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + } + + @Test + public void testTimeUnit() throws IOException { + Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue() / 1000L, + extractor.extractWatermark(split(0))); + } + + @Test + public void testMultipleFiles() throws IOException { + Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); + IcebergSourceSplit combinedSplit = + IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + Assert.assertEquals( + MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); + Assert.assertEquals( + Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), + extractor.extractWatermark(combinedSplit)); + } + + @Test + public void testWrongColumn() { + Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); + Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); + } + + @Test + public void testEmptyStatistics() throws IOException { + Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); + + // Create an extractor for a column we do not have statistics + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(10, "missing_field"); + Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Missing statistics for column"); + } + + private IcebergSourceSplit split(int id) throws IOException { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + ImmutableList.of(TEST_RECORDS.get(id)), + TEMPORARY_FOLDER, + FileFormat.PARQUET, + APPENDER_FACTORY)); + } +} diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index def4f436851b..88234c61123f 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -167,7 +167,12 @@ private IcebergSourceReader createReader( new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), new PlaintextEncryptionManager(), Collections.emptyList()); - return new IcebergSourceReader<>(readerMetrics, readerFunction, splitComparator, readerContext); + return new IcebergSourceReader<>( + SerializableRecordEmitter.defaultEmitter(), + readerMetrics, + readerFunction, + splitComparator, + readerContext); } private static class IdBasedComparator implements SerializableComparator { diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index f85f6277263b..179253cb3a18 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -25,6 +25,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.flink.annotation.Experimental; import org.apache.flink.api.connector.source.Boundedness; @@ -58,15 +59,20 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; +import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -80,6 +86,7 @@ public class IcebergSource implements Source readerFunction; private final SplitAssignerFactory assignerFactory; private final SerializableComparator splitComparator; + private final SerializableRecordEmitter emitter; // Can't use SerializableTable as enumerator needs a regular table // that can discover table changes @@ -91,13 +98,15 @@ public class IcebergSource implements Source readerFunction, SplitAssignerFactory assignerFactory, SerializableComparator splitComparator, - Table table) { + Table table, + SerializableRecordEmitter emitter) { this.tableLoader = tableLoader; this.scanContext = scanContext; this.readerFunction = readerFunction; this.assignerFactory = assignerFactory; this.splitComparator = splitComparator; this.table = table; + this.emitter = emitter; } String name() { @@ -152,7 +161,8 @@ public Boundedness getBoundedness() { public SourceReader createReader(SourceReaderContext readerContext) { IcebergSourceReaderMetrics metrics = new IcebergSourceReaderMetrics(readerContext.metricGroup(), lazyTable().name()); - return new IcebergSourceReader<>(metrics, readerFunction, splitComparator, readerContext); + return new IcebergSourceReader<>( + emitter, metrics, readerFunction, splitComparator, readerContext); } @Override @@ -216,6 +226,8 @@ public static class Builder { private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; + private String watermarkColumn; + private TimeUnit watermarkTimeUnit = TimeUnit.MICROSECONDS; private ReaderFunction readerFunction; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); @@ -237,6 +249,9 @@ public Builder table(Table newTable) { } public Builder assignerFactory(SplitAssignerFactory assignerFactory) { + Preconditions.checkArgument( + watermarkColumn == null, + "Watermark column and SplitAssigner should not be set in the same source"); this.splitAssignerFactory = assignerFactory; return this; } @@ -429,6 +444,33 @@ public Builder setAll(Map properties) { return this; } + /** + * Emits watermarks once per split based on the min value of column statistics from files + * metadata in the given split. The generated watermarks are also used for ordering the splits + * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider + * setting {@link #watermarkTimeUnit(TimeUnit)}. + * + *

Consider setting `read.split.open-file-cost` to prevent combining small files to a single + * split when the watermark is used for watermark alignment. + */ + public Builder watermarkColumn(String columnName) { + Preconditions.checkArgument( + splitAssignerFactory == null, + "Watermark column and SplitAssigner should not be set in the same source"); + this.watermarkColumn = columnName; + return this; + } + + /** + * When the type of the {@link #watermarkColumn} is {@link + * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the + * value. The default value is {@link TimeUnit#MICROSECONDS}. + */ + public Builder watermarkTimeUnit(TimeUnit timeUnit) { + this.watermarkTimeUnit = timeUnit; + return this; + } + /** @deprecated Use {@link #setAll} instead. */ @Deprecated public Builder properties(Map properties) { @@ -453,6 +495,18 @@ public IcebergSource build() { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); } + SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); + if (watermarkColumn != null) { + // Column statistics is needed for watermark generation + contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); + + SplitWatermarkExtractor watermarkExtractor = + new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); + emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); + splitAssignerFactory = + new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); + } + ScanContext context = contextBuilder.build(); if (readerFunction == null) { if (table instanceof BaseMetadataTable) { @@ -485,8 +539,14 @@ public IcebergSource build() { checkRequired(); // Since builder already load the table, pass it to the source to avoid double loading - return new IcebergSource( - tableLoader, context, readerFunction, splitAssignerFactory, splitComparator, table); + return new IcebergSource<>( + tableLoader, + context, + readerFunction, + splitAssignerFactory, + splitComparator, + table, + emitter); } private void checkRequired() { diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..4bb6f0a98c4c --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -0,0 +1,98 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; + +/** + * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics + * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link + * WatermarkExtractorRecordEmitter} along with the actual records. + */ +@Internal +public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { + private final int eventTimeFieldId; + private final String eventTimeFieldName; + private final TimeUnit timeUnit; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param eventTimeFieldName The column which should be used as an event time + * @param timeUnit Used for converting the long value to epoch milliseconds + */ + public ColumnStatsWatermarkExtractor( + Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { + Types.NestedField field = schema.findField(eventTimeFieldName); + TypeID typeID = field.type().typeId(); + Preconditions.checkArgument( + typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), + "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", + typeID); + this.eventTimeFieldId = field.fieldId(); + this.eventTimeFieldName = eventTimeFieldName; + // Use the timeUnit only for Long columns. + this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; + } + + @VisibleForTesting + ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { + this.eventTimeFieldId = eventTimeFieldId; + this.eventTimeFieldName = eventTimeFieldName; + this.timeUnit = TimeUnit.MICROSECONDS; + } + + /** + * Get the watermark for a split using column statistics. + * + * @param split The split + * @return The watermark + * @throws IllegalArgumentException if there is no statistics for the column + */ + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> { + Preconditions.checkArgument( + scanTask.file().lowerBounds() != null + && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, + "Missing statistics for column name = %s in file = %s", + eventTimeFieldName, + eventTimeFieldId, + scanTask.file()); + return timeUnit.toMillis( + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); + }) + .min(Comparator.comparingLong(l -> l)) + .get(); + } +} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java index 8d7d68f961cb..f143b8d2df2e 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java @@ -35,13 +35,14 @@ public class IcebergSourceReader RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { public IcebergSourceReader( + SerializableRecordEmitter emitter, IcebergSourceReaderMetrics metrics, ReaderFunction readerFunction, SerializableComparator splitComparator, SourceReaderContext context) { super( () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), - new IcebergSourceRecordEmitter<>(), + emitter, context.getConfiguration(), context); } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java new file mode 100644 index 000000000000..a6e2c1dae243 --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java @@ -0,0 +1,40 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +@Internal +@FunctionalInterface +public interface SerializableRecordEmitter + extends RecordEmitter, T, IcebergSourceSplit>, Serializable { + static SerializableRecordEmitter defaultEmitter() { + return (element, output, split) -> { + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + }; + } + + static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { + return new WatermarkExtractorRecordEmitter<>(extractor); + } +} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 63% rename from flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java rename to flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java index 337d9d3c4223..d1c50ac8ca52 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceRecordEmitter.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java @@ -18,19 +18,11 @@ */ package org.apache.iceberg.flink.source.reader; -import org.apache.flink.api.connector.source.SourceOutput; -import org.apache.flink.connector.base.source.reader.RecordEmitter; +import java.io.Serializable; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -final class IcebergSourceRecordEmitter - implements RecordEmitter, T, IcebergSourceSplit> { - - IcebergSourceRecordEmitter() {} - - @Override - public void emitRecord( - RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - } +/** The interface used to extract watermarks from splits. */ +public interface SplitWatermarkExtractor extends Serializable { + /** Get the watermark for a split. */ + long extractWatermark(IcebergSourceSplit split); } diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java new file mode 100644 index 000000000000..02ef57d344b1 --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the watermarks, records and updates the split position. + * + *

The Emitter emits watermarks at the beginning of every split provided by the {@link + * SplitWatermarkExtractor}. + */ +class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final SplitWatermarkExtractor timeExtractor; + private String lastSplitId = null; + private long watermark; + + WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { + this.timeExtractor = timeExtractor; + } + + @Override + public void emitRecord( + RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { + if (!split.splitId().equals(lastSplitId)) { + long newWatermark = timeExtractor.extractWatermark(split); + if (newWatermark < watermark) { + LOG.info( + "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", + watermark, + newWatermark, + lastSplitId, + split.splitId()); + } else { + watermark = newWatermark; + output.emitWatermark(new Watermark(watermark)); + LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); + } + + lastSplitId = split.splitId(); + } + + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + } +} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java index 64e03d77debe..56ee92014d12 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.split; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** @@ -45,7 +46,7 @@ public static SerializableComparator fileSequenceNumber() { o1); Preconditions.checkNotNull( seq2, - "IInvalid file sequence number: null. Doesn't support splits written with V1 format: %s", + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", o2); int temp = Long.compare(seq1, seq2); @@ -56,4 +57,20 @@ public static SerializableComparator fileSequenceNumber() { } }; } + + /** Comparator which orders the splits based on watermark of the splits */ + public static SerializableComparator watermark( + SplitWatermarkExtractor watermarkExtractor) { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + long watermark1 = watermarkExtractor.extractWatermark(o1); + long watermark2 = watermarkExtractor.extractWatermark(o2); + + int temp = Long.compare(watermark1, watermark2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 70e7a79d8373..7d991ee603c9 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -39,6 +39,7 @@ import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -97,6 +98,11 @@ protected List generateRecords(int numRecords, long seed) { return RandomGenericData.generate(schema(), numRecords, seed); } + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); + } + @Test public void testBoundedWithTaskManagerFailover() throws Exception { testBoundedIcebergSource(FailoverType.TM); @@ -150,8 +156,7 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio RecordCounterToFail::continueProcessing, miniClusterResource.getMiniCluster()); - SimpleDataUtil.assertTableRecords( - sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } @Test @@ -214,8 +219,7 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - SimpleDataUtil.assertTableRecords( - sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } // ------------------------------------------------------------------------ diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java new file mode 100644 index 000000000000..f7dc931c506c --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; + +public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { + // Increment ts by 15 minutes for each generateRecords batch + private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); + // Within a batch, increment ts by 1 second + private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); + + private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + + @Override + protected IcebergSource.Builder sourceBuilder() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA); + } + + @Override + protected Schema schema() { + return TestFixtures.TS_SCHEMA; + } + + @Override + protected List generateRecords(int numRecords, long seed) { + // Override the ts field to create a more realistic situation for event time alignment + tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); + return RandomGenericData.generate(schema(), numRecords, seed).stream() + .peek( + record -> { + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), + ZoneId.of("Z")); + record.setField("ts", ts); + }) + .collect(Collectors.toList()); + } + + /** + * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves + * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates + * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the + * {@link LocalDateTime} to a Long type so that Comparators can continue to work. + */ + @Override + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> { + SimpleDataUtil.equalsRecords( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + }); + } + + private List convertLocalDateTimeToMilli(List records) { + return records.stream() + .peek( + r -> { + LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); + r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java new file mode 100644 index 000000000000..0bb2eb7766e9 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -0,0 +1,451 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + private static final ConcurrentMap windows = Maps.newConcurrentMap(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + *

    + *
  • - Ordering of the splits + *
  • - Emitting of watermarks + *
  • - Firing windows based on watermarks + *
+ * + *

The test generates 4 splits + * + *

    + *
  • - Split 1 - Watermark 100 min + *
  • - Split 2, 3 - Watermark 0 min + *
  • - Split 4 - Watermark 6 min + *
+ * + *

Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + *

Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + *

    + *
  • - Split 2, 3 + *
  • - Split 4 + *
  • - Split 1 + *
+ * + *

As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + *

Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction() { + @Override + public void apply( + TimeWindow window, Iterable values, Collector out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicInteger count = new AtomicInteger(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + windows.put(window.getStart(), count.get()); + } + }); + + // Use static variable to collect the windows, since other solutions were flaky + windows.clear(); + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of( + 0L, + RECORD_NUM_FOR_2_SPLITS, + TimeUnit.MINUTES.toMillis(5), + 2, + TimeUnit.MINUTES.toMillis(100), + 3))); + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + *

    + *
  • - Emitting of watermarks + *
  • - Watermark alignment + *
+ * + *

The test generates 3 splits + * + *

    + *
  • - Split 1 - Watermark 100 min + *
  • - Split 2, 3 - Watermark 0 min + *
+ * + * The splits are read in the following order: + * + *
    + *
  • - Split 2, 3 (Task Manager 1, Task Manager 2) + *
  • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + *
+ * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + *

The status of the watermark alignment is checked by the alignment related metrics. + * + *

Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + *

After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + try (CloseableIterator resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); + + // Check that the read the non-blocked data + // The first RECORD_NUM_FOR_2_SPLITS should be read + // 1 or more from the runaway reader should be arrived depending on thread scheduling + waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + List newBatch1 = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + List newBatch2 = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + dataAppender.appendToTable( + dataAppender.writeFile(newBatch1), dataAppender.writeFile(newBatch2)); + // The records received will highly depend on scheduling + // We minimally get 3 records from the non-blocked reader + // We might get 1 record from the blocked reader (as part of the previous batch - + // file_1-recordTs_100) + // We might get 3 records form the non-blocked reader if it gets both new splits + waitForRecords(resultIterator, 3); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); + + // Add some new records which should unblock the throttled reader + batch = + ImmutableList.of( + generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); + dataAppender.appendToTable(batch); + // We should get all the records at this point + waitForRecords(resultIterator, 6); + + // Wait for the new drift to decrease below the allowed drift to signal the normal state + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); + } + } + + protected IcebergSource source() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA) + .splitSize(100L) + .streaming(true) + .monitorInterval(Duration.ofMillis(2)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + } + + protected Record generateRecord(int minutes, String str) { + // Override the ts field to create a more realistic situation for event time alignment + Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), + ZoneId.of("Z")); + record.setField("ts", ts); + record.setField("str", str); + return record; + } + + protected void assertRecords( + Collection expectedRecords, CloseableIterator iterator) throws Exception { + Set expected = + expectedRecords.stream() + .map(e -> RowDataConverter.convert(TestFixtures.TS_SCHEMA, e)) + .collect(Collectors.toSet()); + Assert.assertEquals(expected, waitForRecords(iterator, expectedRecords.size())); + } + + protected Set waitForRecords(CloseableIterator iterator, int num) { + Set received = Sets.newHashSetWithExpectedSize(num); + assertThat( + CompletableFuture.supplyAsync( + () -> { + int count = 0; + while (count < num && iterator.hasNext()) { + received.add(iterator.next()); + count++; + } + + if (count < num) { + throw new IllegalStateException(String.format("Fail to get %d records.", num)); + } + + return true; + })) + .succeedsWithin(DEFAULT_COLLECT_DATA_TIMEOUT); + + return received; + } + + private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { + String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; + return reporter.findMetrics(jobID, metricsName).values().stream() + .map(m -> (Gauge) m) + .filter(m -> m.getValue() == withValue) + .findFirst(); + } + + private GenericAppenderHelper appender() { + // We need to create multiple splits, so we need to generate parquet files with multiple offsets + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("write.parquet.page-size-bytes", "64"); + hadoopConf.set("write.parquet.row-group-size-bytes", "64"); + return new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + } + + private static RowData row(long time, long count) { + GenericRowData result = new GenericRowData(2); + result.setField(0, time); + result.setField(1, String.valueOf(count)); + return result; + } + + private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { + @Override + public long extractTimestamp(RowData element, long recordTimestamp) { + return element.getTimestamp(0, 0).getMillisecond(); + } + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index f28677ca9d6a..090b304942c6 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -43,15 +43,13 @@ public void testEmptyInitialization() { @Test public void testStaticEnumeratorSequence() throws Exception { SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 1)); + assigner.onDiscoveredSplits(createSplits(4, 1, "1")); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); - assigner.onUnassignedSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1)); + assigner.onUnassignedSplits(createSplits(1, 1, "1")); assertSnapshot(assigner, 2); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -66,15 +64,12 @@ public void testContinuousEnumeratorSequence() throws Exception { SplitAssigner assigner = splitAssigner(); assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - List splits1 = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + List splits1 = createSplits(1, 1, "1"); assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); - List splits2 = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + List splits2 = createSplits(1, 1, "1"); assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 2, 1)); + assigner.onDiscoveredSplits(createSplits(2, 1, "1")); assertSnapshot(assigner, 2); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -125,5 +120,11 @@ protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Assert.assertEquals(splitCount, stateBeforeGet.size()); } + protected List createSplits(int fileCount, int filesPerSplit, String version) + throws Exception { + return SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, fileCount, filesPerSplit, version); + } + protected abstract SplitAssigner splitAssigner(); } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java index 8b9e132e0e22..e78634e6b873 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.iceberg.ContentFile; -import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; @@ -40,9 +39,7 @@ protected SplitAssigner splitAssigner() { public void testMultipleFilesInAnIcebergSplit() { SplitAssigner assigner = splitAssigner(); Assertions.assertThatThrownBy( - () -> - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2, "2")), + () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Please use 'split-open-file-cost'"); @@ -52,8 +49,7 @@ public void testMultipleFilesInAnIcebergSplit() { @Test public void testSplitSort() throws Exception { SplitAssigner assigner = splitAssigner(); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 5, 1, "2"); + List splits = createSplits(5, 1, "2"); assigner.onDiscoveredSplits(splits.subList(3, 5)); assigner.onDiscoveredSplits(splits.subList(0, 1)); @@ -76,7 +72,7 @@ public void testSerializable() { Assert.assertNotNull(comparator); } - protected void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { GetSplitResult result = assigner.getNext(null); ContentFile file = result.split().task().files().iterator().next().file(); Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java new file mode 100644 index 000000000000..e1fc63fda918 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SerializationUtil; +import org.junit.Assert; +import org.junit.Test; + +public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { + public static final Schema SCHEMA = + new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) + .createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 2, "2")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() { + SplitAssigner assigner = splitAssigner(); + + Instant now = Instant.now(); + List splits = + IntStream.range(0, 5) + .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) + .collect(Collectors.toList()); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, splits.get(0)); + assertGetNext(assigner, splits.get(1)); + assertGetNext(assigner, splits.get(2)); + assertGetNext(assigner, splits.get(3)); + assertGetNext(assigner, splits.get(4)); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = + SerializationUtil.serializeToBytes( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor( + TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { + GetSplitResult result = assigner.getNext(null); + Assert.assertEquals(result.split(), split); + } + + @Override + protected List createSplits( + int fileCount, int filesPerSplit, String version) { + return IntStream.range(0, fileCount / filesPerSplit) + .mapToObj( + splitNum -> + splitFromRecords( + IntStream.range(0, filesPerSplit) + .mapToObj( + fileNum -> + RandomGenericData.generate( + SCHEMA, 2, splitNum * filesPerSplit + fileNum)) + .collect(Collectors.toList()))) + .collect(Collectors.toList()); + } + + private IcebergSourceSplit splitFromInstant(Instant instant) { + Record record = GenericRecord.create(SCHEMA); + record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); + return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); + } + + private IcebergSourceSplit splitFromRecords(List> records) { + try { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + } catch (IOException e) { + throw new RuntimeException("Split creation exception", e); + } + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index f9ceaf842263..2a2503ef2478 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -60,9 +60,12 @@ public static FileScanTask createFileTask( FileFormat fileFormat, FileAppenderFactory appenderFactory) throws IOException { - try (FileAppender appender = - appenderFactory.newAppender(Files.localOutput(file), fileFormat)) { + FileAppender appender = + appenderFactory.newAppender(Files.localOutput(file), fileFormat); + try { appender.addAll(records); + } finally { + appender.close(); } DataFile dataFile = @@ -71,6 +74,7 @@ public static FileScanTask createFileTask( .withFileSizeInBytes(file.length()) .withPath(file.toString()) .withFormat(fileFormat) + .withMetrics(appender.metrics()) .build(); ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..afe8a5d0152c --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -0,0 +1,178 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestColumnStatsWatermarkExtractor { + public static final Schema SCHEMA = + new Schema( + required(1, "timestamp_column", Types.TimestampType.withoutZone()), + required(2, "timestamptz_column", Types.TimestampType.withZone()), + required(3, "long_column", Types.LongType.get()), + required(4, "string_column", Types.StringType.get())); + + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + private static final List> TEST_RECORDS = + ImmutableList.of( + RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); + + private static final List> MIN_VALUES = + ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); + + private final String columnName; + + @BeforeClass + public static void updateMinValue() { + for (int i = 0; i < TEST_RECORDS.size(); ++i) { + for (Record r : TEST_RECORDS.get(i)) { + Map minValues = MIN_VALUES.get(i); + + LocalDateTime localDateTime = (LocalDateTime) r.get(0); + minValues.merge( + "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); + + OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); + minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); + + minValues.merge("long_column", (Long) r.get(2), Math::min); + } + } + } + + @Parameterized.Parameters(name = "{0}") + public static Collection data() { + return ImmutableList.of( + new Object[] {"timestamp_column"}, + new Object[] {"timestamptz_column"}, + new Object[] {"long_column"}); + } + + public TestColumnStatsWatermarkExtractor(String columnName) { + this.columnName = columnName; + } + + @Test + public void testSingle() throws IOException { + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + } + + @Test + public void testTimeUnit() throws IOException { + Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue() / 1000L, + extractor.extractWatermark(split(0))); + } + + @Test + public void testMultipleFiles() throws IOException { + Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); + IcebergSourceSplit combinedSplit = + IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + Assert.assertEquals( + MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); + Assert.assertEquals( + Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), + extractor.extractWatermark(combinedSplit)); + } + + @Test + public void testWrongColumn() { + Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); + Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); + } + + @Test + public void testEmptyStatistics() throws IOException { + Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); + + // Create an extractor for a column we do not have statistics + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(10, "missing_field"); + Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Missing statistics for column"); + } + + private IcebergSourceSplit split(int id) throws IOException { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + ImmutableList.of(TEST_RECORDS.get(id)), + TEMPORARY_FOLDER, + FileFormat.PARQUET, + APPENDER_FACTORY)); + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index def4f436851b..88234c61123f 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -167,7 +167,12 @@ private IcebergSourceReader createReader( new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), new PlaintextEncryptionManager(), Collections.emptyList()); - return new IcebergSourceReader<>(readerMetrics, readerFunction, splitComparator, readerContext); + return new IcebergSourceReader<>( + SerializableRecordEmitter.defaultEmitter(), + readerMetrics, + readerFunction, + splitComparator, + readerContext); } private static class IdBasedComparator implements SerializableComparator { From d2ab70927c9509b10b858845bfac8c4ebb9acfec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 29 Nov 2023 00:14:28 +0100 Subject: [PATCH 72/94] Build: Bump nessie from 0.73.0 to 0.74.0 (#9153) Bumps `nessie` from 0.73.0 to 0.74.0. Updates `org.projectnessie.nessie:nessie-client` from 0.73.0 to 0.74.0 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.73.0 to 0.74.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory` from 0.73.0 to 0.74.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.73.0 to 0.74.0 --- updated-dependencies: - dependency-name: org.projectnessie.nessie:nessie-client dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-jaxrs-testextension dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-inmemory dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.projectnessie.nessie:nessie-versioned-storage-testextension dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 1ee887d34520..46d7a6cfa426 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -64,7 +64,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.73.0" +nessie = "0.74.0" netty-buffer = "4.1.97.Final" netty-buffer-compat = "4.1.68.Final" object-client-bundle = "3.3.2" From d247b20f166ccb0b92443d4b05330b1e0d9c5d49 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 28 Nov 2023 15:52:55 -0800 Subject: [PATCH 73/94] Core: Remove deprecated code in DeleteFileIndex (#9166) --- .../org/apache/iceberg/DeleteFileIndex.java | 41 ------------------- 1 file changed, 41 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index 51917a71e9d9..02e55c595f3c 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -78,16 +78,6 @@ class DeleteFileIndex { private final boolean isEmpty; private final boolean useColumnStatsFiltering; - /** @deprecated since 1.4.0, will be removed in 1.5.0. */ - @Deprecated - DeleteFileIndex( - Map specs, - long[] globalSeqs, - DeleteFile[] globalDeletes, - Map, Pair> deletesByPartition) { - this(specs, index(specs, globalSeqs, globalDeletes), index(specs, deletesByPartition), true); - } - private DeleteFileIndex( Map specs, DeleteFileGroup globalDeletes, @@ -374,37 +364,6 @@ private static boolean containsNull(Map nullValueCounts, Types.Ne return nullValueCount > 0; } - private static DeleteFileGroup index( - Map specs, Pair pairs) { - return index(specs, pairs.first(), pairs.second()); - } - - private static DeleteFileGroup index( - Map specs, long[] seqs, DeleteFile[] files) { - if (files == null || files.length == 0) { - return null; - } - - IndexedDeleteFile[] indexedGlobalDeleteFiles = new IndexedDeleteFile[files.length]; - - for (int pos = 0; pos < files.length; pos++) { - DeleteFile file = files[pos]; - PartitionSpec spec = specs.get(file.specId()); - long applySequenceNumber = seqs[pos]; - indexedGlobalDeleteFiles[pos] = new IndexedDeleteFile(spec, file, applySequenceNumber); - } - - return new DeleteFileGroup(seqs, indexedGlobalDeleteFiles); - } - - private static Map, DeleteFileGroup> index( - Map specs, - Map, Pair> deletesByPartition) { - Map, DeleteFileGroup> indexed = Maps.newHashMap(); - deletesByPartition.forEach((key, value) -> indexed.put(key, index(specs, value))); - return indexed; - } - static Builder builderFor(FileIO io, Iterable deleteManifests) { return new Builder(io, Sets.newHashSet(deleteManifests)); } From ac71ceaa8d19405da4a4018a90ade8717c2a63c8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 30 Nov 2023 21:04:38 -0800 Subject: [PATCH 74/94] Build: Bump mkdocs-material from 9.4.10 to 9.4.12 (#9159) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.4.10 to 9.4.12. - [Release notes](https://github.com/squidfunk/mkdocs-material/releases) - [Changelog](https://github.com/squidfunk/mkdocs-material/blob/master/CHANGELOG) - [Commits](https://github.com/squidfunk/mkdocs-material/compare/9.4.10...9.4.12) --- updated-dependencies: - dependency-name: mkdocs-material dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- site/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/requirements.txt b/site/requirements.txt index 6b5850500871..33b14fe4f540 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.4.10 +mkdocs-material==9.4.12 mkdocs-material-extensions==1.3 mkdocs-monorepo-plugin==1.0.5 mkdocs-redirects==1.2.1 From de2505027e82962fc7c48623c9e1a1e85a596f1b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 30 Nov 2023 21:06:09 -0800 Subject: [PATCH 75/94] Build: Bump software.amazon.awssdk:bom from 2.21.26 to 2.21.29 (#9154) Bumps software.amazon.awssdk:bom from 2.21.26 to 2.21.29. --- updated-dependencies: - dependency-name: software.amazon.awssdk:bom dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 46d7a6cfa426..f81c2e36bec7 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -27,7 +27,7 @@ arrow = "14.0.1" avro = "1.11.3" assertj-core = "3.24.2" awaitility = "4.2.0" -awssdk-bom = "2.21.26" +awssdk-bom = "2.21.29" azuresdk-bom = "1.2.18" caffeine = "2.9.3" calcite = "1.10.0" From 09a8ad5c391d4e1b1027a02b7e2d366d0ff56fc2 Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Sat, 2 Dec 2023 13:16:07 -0800 Subject: [PATCH 76/94] Core: REST HttpClient connections config (#9195) --- .../org/apache/iceberg/rest/HTTPClient.java | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java index e1e5637f231b..55ed1422b3e1 100644 --- a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java @@ -34,6 +34,8 @@ import org.apache.hc.client5.http.impl.classic.CloseableHttpResponse; import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; import org.apache.hc.client5.http.impl.classic.HttpClients; +import org.apache.hc.client5.http.impl.io.PoolingHttpClientConnectionManagerBuilder; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; import org.apache.hc.core5.http.ContentType; import org.apache.hc.core5.http.Header; import org.apache.hc.core5.http.HttpHeaders; @@ -72,6 +74,10 @@ public class HTTPClient implements RESTClient { static final String CLIENT_GIT_COMMIT_SHORT_HEADER = "X-Client-Git-Commit-Short"; private static final String REST_MAX_RETRIES = "rest.client.max-retries"; + private static final String REST_MAX_CONNECTIONS = "rest.client.max-connections"; + private static final int REST_MAX_CONNECTIONS_DEFAULT = 100; + private static final String REST_MAX_CONNECTIONS_PER_ROUTE = "rest.client.connections-per-route"; + private static final int REST_MAX_CONNECTIONS_PER_ROUTE_DEFAULT = 100; private final String uri; private final CloseableHttpClient httpClient; @@ -88,6 +94,18 @@ private HTTPClient( HttpClientBuilder clientBuilder = HttpClients.custom(); + HttpClientConnectionManager connectionManager = + PoolingHttpClientConnectionManagerBuilder.create() + .useSystemProperties() + .setMaxConnTotal(Integer.getInteger(REST_MAX_CONNECTIONS, REST_MAX_CONNECTIONS_DEFAULT)) + .setMaxConnPerRoute( + PropertyUtil.propertyAsInt( + properties, + REST_MAX_CONNECTIONS_PER_ROUTE, + REST_MAX_CONNECTIONS_PER_ROUTE_DEFAULT)) + .build(); + clientBuilder.setConnectionManager(connectionManager); + if (baseHeaders != null) { clientBuilder.setDefaultHeaders( baseHeaders.entrySet().stream() From 9bd62f79f8cd973c39d14e89163cb1c707470ed2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 3 Dec 2023 18:15:20 +0100 Subject: [PATCH 77/94] Build: Bump org.apache.httpcomponents.client5:httpclient5 (#9202) Bumps [org.apache.httpcomponents.client5:httpclient5](https://github.com/apache/httpcomponents-client) from 5.2.1 to 5.2.3. - [Changelog](https://github.com/apache/httpcomponents-client/blob/rel/v5.2.3/RELEASE_NOTES.txt) - [Commits](https://github.com/apache/httpcomponents-client/compare/rel/v5.2.1...rel/v5.2.3) --- updated-dependencies: - dependency-name: org.apache.httpcomponents.client5:httpclient5 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index f81c2e36bec7..d9ca188e9384 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -43,7 +43,7 @@ google-libraries-bom = "26.27.0" guava = "32.1.3-jre" hadoop2 = "2.7.3" hadoop3-client = "3.3.6" -httpcomponents-httpclient5 = "5.2.1" +httpcomponents-httpclient5 = "5.2.3" hive2 = { strictly = "[2, 3[", prefer = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" immutables-value = "2.10.0" From 1ed1b4ba965462698ec24bd2920d29195e2acae4 Mon Sep 17 00:00:00 2001 From: emkornfield Date: Mon, 4 Dec 2023 11:39:27 -0800 Subject: [PATCH 78/94] Spec: Clarify partition equality (#9125) Co-authored-by: Fokko Driesprong --- format/spec.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index 27e2762f7724..5d6dded5ee76 100644 --- a/format/spec.md +++ b/format/spec.md @@ -305,6 +305,9 @@ The source column, selected by id, must be a primitive type and cannot be contai Partition specs capture the transform from table data to partition values. This is used to transform predicates to partition predicates, in addition to transforming data values. Deriving partition predicates from column predicates on the table data is used to separate the logical queries from physical storage: the partitioning can change and the correct partition filters are always derived from column predicates. This simplifies queries because users don’t have to supply both logical predicates and partition predicates. For more information, see Scan Planning below. +Two partition specs are considered equivalent with each other if they have the same number of fields and for each corresponding field, the fields have the same source column ID, transform definition and partition name. Writers must not create a new parition spec if there already exists a compatible partition spec defined in the table. + +Partition field IDs must be reused if an existing partition spec contains an equivalent field. #### Partition Transforms @@ -595,7 +598,7 @@ Delete files that match the query filter must be applied to data files at read t - The data file's partition (both spec and partition values) is equal to the delete file's partition * An _equality_ delete file must be applied to a data file when all of the following are true: - The data file's data sequence number is _strictly less than_ the delete's data sequence number - - The data file's partition (both spec and partition values) is equal to the delete file's partition _or_ the delete file's partition spec is unpartitioned + - The data file's partition (both spec id and partition values) is equal to the delete file's partition _or_ the delete file's partition spec is unpartitioned In general, deletes are applied only to data files that are older and in the same partition, except for two special cases: @@ -607,6 +610,7 @@ Notes: 1. An alternative, *strict projection*, creates a partition predicate that will match a file if all of the rows in the file must match the scan predicate. These projections are used to calculate the residual predicates for each file in a scan. 2. For example, if `file_a` has rows with `id` between 1 and 10 and a delete file contains rows with `id` between 1 and 4, a scan for `id = 9` may ignore the delete file because none of the deletes can match a row that will be selected. +3. Floating point partition values are considered equal if their IEEE 754 floating-point "single format" bit layout are equal with NaNs normalized to have only the the most significant mantissa bit set (the equivelant of calling `Float.floatToIntBits` or `Double.doubleToLongBits` in Java). The Avro specification requires all floating point values to be encoded in this format. #### Snapshot Reference From 99843f03efecc3f5ef7d3e1d32aae5eff22cb315 Mon Sep 17 00:00:00 2001 From: Andrew Sherman Date: Mon, 4 Dec 2023 14:46:37 -0800 Subject: [PATCH 79/94] Core: Expired Snapshot files in a transaction should be deleted. (#9183) When a snapshot is expired as part of a transaction, the manifest list file(s) should be deleted when the transaction commits. A recent change (#6634) ensured that these files are not deleted when they have also been committed as part of a transaction, but this breaks the simple case where no new files are committed. Fix this by not skipping deletion when the list of committed files is empty. TESTING: Extended a unit test to ensure that manifest list files are deleted. Ran the test without the fix on a branch where #6634 was reverted to show that this is a regression. --- .../org/apache/iceberg/BaseTransaction.java | 24 ++++++++----------- .../org/apache/iceberg/TableTestBase.java | 9 +++++++ .../apache/iceberg/TestRemoveSnapshots.java | 7 +++++- .../iceberg/TestSequenceNumberForV2Table.java | 8 +++++++ 4 files changed, 33 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index 32cf695c8b5a..018f70eb16fa 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -446,20 +446,16 @@ private void commitSimpleTransaction() { } Set committedFiles = committedFiles(ops, newSnapshots); - if (committedFiles != null) { - // delete all of the files that were deleted in the most recent set of operation commits - Tasks.foreach(deletedFiles) - .suppressFailureWhenFinished() - .onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc)) - .run( - path -> { - if (!committedFiles.contains(path)) { - ops.io().deleteFile(path); - } - }); - } else { - LOG.warn("Failed to load metadata for a committed snapshot, skipping clean-up"); - } + // delete all of the files that were deleted in the most recent set of operation commits + Tasks.foreach(deletedFiles) + .suppressFailureWhenFinished() + .onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc)) + .run( + path -> { + if (committedFiles == null || !committedFiles.contains(path)) { + ops.io().deleteFile(path); + } + }); } catch (RuntimeException e) { LOG.warn("Failed to load committed metadata, skipping clean-up", e); diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index 68ce05528964..c3db85910138 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -212,6 +212,15 @@ List listManifestFiles(File tableDirToList) { && Files.getFileExtension(name).equalsIgnoreCase("avro"))); } + List listManifestLists(String tableDirToList) { + return Lists.newArrayList( + new File(tableDirToList, "metadata") + .listFiles( + (dir, name) -> + name.startsWith("snap") + && Files.getFileExtension(name).equalsIgnoreCase("avro"))); + } + public static long countAllMetadataFiles(File tableDir) { return Arrays.stream(new File(tableDir, "metadata").listFiles()) .filter(f -> f.isFile()) diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index 71455c571282..fc3aa6c91685 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -440,7 +440,10 @@ public void testRetainNAvailableSnapshotsWithTransaction() { t3 = System.currentTimeMillis(); } - // Retain last 2 snapshots + Assert.assertEquals( + "Should be 3 manifest lists", 3, listManifestLists(table.location()).size()); + + // Retain last 2 snapshots, which means 1 is deleted. Transaction tx = table.newTransaction(); removeSnapshots(tx.table()).expireOlderThan(t3).retainLast(2).commit(); tx.commitTransaction(); @@ -449,6 +452,8 @@ public void testRetainNAvailableSnapshotsWithTransaction() { "Should have two snapshots.", 2, Lists.newArrayList(table.snapshots()).size()); Assert.assertEquals( "First snapshot should not present.", null, table.snapshot(firstSnapshotId)); + Assert.assertEquals( + "Should be 2 manifest lists", 2, listManifestLists(table.location()).size()); } @Test diff --git a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java index 08bdc64a0853..86842b681278 100644 --- a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java +++ b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java @@ -309,6 +309,8 @@ public void testExpirationInTransaction() { V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); V2Assert.assertEquals( "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); + V2Assert.assertEquals( + "Should be 1 manifest list", 1, listManifestLists(table.location()).size()); table.newAppend().appendFile(FILE_B).commit(); Snapshot snap2 = table.currentSnapshot(); @@ -319,12 +321,18 @@ public void testExpirationInTransaction() { V2Assert.assertEquals("Snapshot sequence number should be 2", 2, snap2.sequenceNumber()); V2Assert.assertEquals( "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber()); + V2Assert.assertEquals( + "Should be 2 manifest lists", 2, listManifestLists(table.location()).size()); Transaction txn = table.newTransaction(); txn.expireSnapshots().expireSnapshotId(commitId1).commit(); txn.commitTransaction(); V2Assert.assertEquals( "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber()); + V2Assert.assertEquals( + "Should be 1 manifest list as 1 was deleted", + 1, + listManifestLists(table.location()).size()); } @Test From a4d47567e1fef44f4443250537f09dc73a1f7583 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 5 Dec 2023 12:03:43 +0100 Subject: [PATCH 80/94] Core: Schema for a branch should return table schema (#9131) When retrieving the schema for branch we should always return the table schema instead of the snapshot schema. This is because the table schema is the schema that will be used when the branch will be created. We should only return the schema of the snapshot when we have a tag. --- .../org/apache/iceberg/util/SnapshotUtil.java | 40 +++---- .../apache/iceberg/util/TestSnapshotUtil.java | 65 +++++++++++ .../spark/source/TestSnapshotSelection.java | 106 ++++++++++++++++-- .../spark/source/TestSnapshotSelection.java | 106 ++++++++++++++++-- .../spark/source/TestSnapshotSelection.java | 106 ++++++++++++++++-- 5 files changed, 380 insertions(+), 43 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java index 75d44936913a..b0ec879bda8d 100644 --- a/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java @@ -409,49 +409,51 @@ public static Schema schemaFor(Table table, Long snapshotId, Long timestampMilli } /** - * Return the schema of the snapshot at a given branch. + * Return the schema of the snapshot at a given ref. * - *

If branch does not exist, the table schema is returned because it will be the schema when - * the new branch is created. + *

If the ref does not exist or the ref is a branch, the table schema is returned because it + * will be the schema when the new branch is created. If the ref is a tag, then the snapshot + * schema is returned. * * @param table a {@link Table} - * @param branch branch name of the table (nullable) - * @return schema of the specific snapshot at the given branch + * @param ref ref name of the table (nullable) + * @return schema of the specific snapshot at the given ref */ - public static Schema schemaFor(Table table, String branch) { - if (branch == null || branch.equals(SnapshotRef.MAIN_BRANCH)) { + public static Schema schemaFor(Table table, String ref) { + if (ref == null || ref.equals(SnapshotRef.MAIN_BRANCH)) { return table.schema(); } - Snapshot ref = table.snapshot(branch); - if (ref == null) { + SnapshotRef snapshotRef = table.refs().get(ref); + if (null == snapshotRef || snapshotRef.isBranch()) { return table.schema(); } - return schemaFor(table, ref.snapshotId()); + return schemaFor(table, snapshotRef.snapshotId()); } /** - * Return the schema of the snapshot at a given branch. + * Return the schema of the snapshot at a given ref. * - *

If branch does not exist, the table schema is returned because it will be the schema when - * the new branch is created. + *

If the ref does not exist or the ref is a branch, the table schema is returned because it + * will be the schema when the new branch is created. If the ref is a tag, then the snapshot + * schema is returned. * * @param metadata a {@link TableMetadata} - * @param branch branch name of the table (nullable) + * @param ref ref name of the table (nullable) * @return schema of the specific snapshot at the given branch */ - public static Schema schemaFor(TableMetadata metadata, String branch) { - if (branch == null || branch.equals(SnapshotRef.MAIN_BRANCH)) { + public static Schema schemaFor(TableMetadata metadata, String ref) { + if (ref == null || ref.equals(SnapshotRef.MAIN_BRANCH)) { return metadata.schema(); } - SnapshotRef ref = metadata.ref(branch); - if (ref == null) { + SnapshotRef snapshotRef = metadata.ref(ref); + if (snapshotRef == null || snapshotRef.isBranch()) { return metadata.schema(); } - Snapshot snapshot = metadata.snapshot(ref.snapshotId()); + Snapshot snapshot = metadata.snapshot(snapshotRef.snapshotId()); return metadata.schemas().get(snapshot.schemaId()); } diff --git a/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java b/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java index 576df47e7b2f..db6be5bcfe48 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.util; +import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; @@ -187,4 +188,68 @@ private void expectedSnapshots(long[] snapshotIdExpected, Iterable sna .toArray(); assertThat(actualSnapshots).isEqualTo(snapshotIdExpected); } + + @Test + public void schemaForRef() { + Schema initialSchema = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get())); + assertThat(table.schema().asStruct()).isEqualTo(initialSchema.asStruct()); + + assertThat(SnapshotUtil.schemaFor(table, null).asStruct()).isEqualTo(initialSchema.asStruct()); + assertThat(SnapshotUtil.schemaFor(table, "non-existing-ref").asStruct()) + .isEqualTo(initialSchema.asStruct()); + assertThat(SnapshotUtil.schemaFor(table, SnapshotRef.MAIN_BRANCH).asStruct()) + .isEqualTo(initialSchema.asStruct()); + } + + @Test + public void schemaForBranch() { + Schema initialSchema = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get())); + assertThat(table.schema().asStruct()).isEqualTo(initialSchema.asStruct()); + + String branch = "branch"; + table.manageSnapshots().createBranch(branch).commit(); + + assertThat(SnapshotUtil.schemaFor(table, branch).asStruct()) + .isEqualTo(initialSchema.asStruct()); + + table.updateSchema().addColumn("zip", Types.IntegerType.get()).commit(); + Schema expected = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + optional(3, "zip", Types.IntegerType.get())); + + assertThat(table.schema().asStruct()).isEqualTo(expected.asStruct()); + assertThat(SnapshotUtil.schemaFor(table, branch).asStruct()).isEqualTo(expected.asStruct()); + } + + @Test + public void schemaForTag() { + Schema initialSchema = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get())); + assertThat(table.schema().asStruct()).isEqualTo(initialSchema.asStruct()); + + String tag = "tag"; + table.manageSnapshots().createTag(tag, table.currentSnapshot().snapshotId()).commit(); + + assertThat(SnapshotUtil.schemaFor(table, tag).asStruct()).isEqualTo(initialSchema.asStruct()); + + table.updateSchema().addColumn("zip", Types.IntegerType.get()).commit(); + Schema expected = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + optional(3, "zip", Types.IntegerType.get())); + + assertThat(table.schema().asStruct()).isEqualTo(expected.asStruct()); + assertThat(SnapshotUtil.schemaFor(table, tag).asStruct()).isEqualTo(initialSchema.asStruct()); + } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java index 276fbcd592ae..a161224275ca 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java @@ -31,10 +31,12 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.SparkSession; import org.assertj.core.api.Assertions; import org.junit.AfterClass; @@ -402,16 +404,104 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { // Deleting a column to indicate schema change table.updateSchema().deleteColumn("data").commit(); - // The data should have the deleted column as it was captured in an earlier snapshot. - Dataset deletedColumnBranchSnapshotResult = + // The data should not have the deleted column + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .collectAsList()) + .containsExactly(RowFactory.create(1), RowFactory.create(2), RowFactory.create(3)); + + // re-introducing the column should not let the data re-appear + table.updateSchema().addColumn("data", Types.StringType.get()).commit(); + + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList()) + .containsExactly( + new SimpleRecord(1, null), new SimpleRecord(2, null), new SimpleRecord(3, null)); + } + + @Test + public void testWritingToBranchAfterSchemaChange() throws IOException { + String tableLocation = temp.newFolder("iceberg-table").toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + + Dataset branchSnapshotResult = spark.read().format("iceberg").option("branch", "branch").load(tableLocation); - List deletedColumnBranchSnapshotRecords = - deletedColumnBranchSnapshotResult - .orderBy("id") - .as(Encoders.bean(SimpleRecord.class)) - .collectAsList(); + List branchSnapshotRecords = + branchSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, deletedColumnBranchSnapshotRecords); + "Current snapshot rows should match", expectedRecords, branchSnapshotRecords); + + // Deleting and add a new column of the same type to indicate schema change + table.updateSchema().deleteColumn("data").addColumn("zip", Types.IntegerType.get()).commit(); + + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .collectAsList()) + .containsExactly( + RowFactory.create(1, null), RowFactory.create(2, null), RowFactory.create(3, null)); + + // writing new records into the branch should work with the new column + List records = + Lists.newArrayList( + RowFactory.create(4, 12345), RowFactory.create(5, 54321), RowFactory.create(6, 67890)); + + Dataset dataFrame = + spark.createDataFrame( + records, + SparkSchemaUtil.convert( + new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(2, "zip", Types.IntegerType.get())))); + dataFrame + .select("id", "zip") + .write() + .format("iceberg") + .option("branch", "branch") + .mode("append") + .save(tableLocation); + + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .collectAsList()) + .hasSize(6) + .contains( + RowFactory.create(1, null), RowFactory.create(2, null), RowFactory.create(3, null)) + .containsAll(records); } @Test diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java index f1374c050d5c..645afd4542e4 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java @@ -37,10 +37,12 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.SparkSession; import org.assertj.core.api.Assertions; import org.junit.AfterClass; @@ -425,16 +427,104 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { // Deleting a column to indicate schema change table.updateSchema().deleteColumn("data").commit(); - // The data should have the deleted column as it was captured in an earlier snapshot. - Dataset deletedColumnBranchSnapshotResult = + // The data should not have the deleted column + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .collectAsList()) + .containsExactly(RowFactory.create(1), RowFactory.create(2), RowFactory.create(3)); + + // re-introducing the column should not let the data re-appear + table.updateSchema().addColumn("data", Types.StringType.get()).commit(); + + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList()) + .containsExactly( + new SimpleRecord(1, null), new SimpleRecord(2, null), new SimpleRecord(3, null)); + } + + @Test + public void testWritingToBranchAfterSchemaChange() throws IOException { + String tableLocation = temp.newFolder("iceberg-table").toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + + Dataset branchSnapshotResult = spark.read().format("iceberg").option("branch", "branch").load(tableLocation); - List deletedColumnBranchSnapshotRecords = - deletedColumnBranchSnapshotResult - .orderBy("id") - .as(Encoders.bean(SimpleRecord.class)) - .collectAsList(); + List branchSnapshotRecords = + branchSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, deletedColumnBranchSnapshotRecords); + "Current snapshot rows should match", expectedRecords, branchSnapshotRecords); + + // Deleting and add a new column of the same type to indicate schema change + table.updateSchema().deleteColumn("data").addColumn("zip", Types.IntegerType.get()).commit(); + + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .collectAsList()) + .containsExactly( + RowFactory.create(1, null), RowFactory.create(2, null), RowFactory.create(3, null)); + + // writing new records into the branch should work with the new column + List records = + Lists.newArrayList( + RowFactory.create(4, 12345), RowFactory.create(5, 54321), RowFactory.create(6, 67890)); + + Dataset dataFrame = + spark.createDataFrame( + records, + SparkSchemaUtil.convert( + new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(2, "zip", Types.IntegerType.get())))); + dataFrame + .select("id", "zip") + .write() + .format("iceberg") + .option("branch", "branch") + .mode("append") + .save(tableLocation); + + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .collectAsList()) + .hasSize(6) + .contains( + RowFactory.create(1, null), RowFactory.create(2, null), RowFactory.create(3, null)) + .containsAll(records); } @Test diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java index f1374c050d5c..645afd4542e4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java @@ -37,10 +37,12 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkReadOptions; +import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.SparkSession; import org.assertj.core.api.Assertions; import org.junit.AfterClass; @@ -425,16 +427,104 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { // Deleting a column to indicate schema change table.updateSchema().deleteColumn("data").commit(); - // The data should have the deleted column as it was captured in an earlier snapshot. - Dataset deletedColumnBranchSnapshotResult = + // The data should not have the deleted column + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .collectAsList()) + .containsExactly(RowFactory.create(1), RowFactory.create(2), RowFactory.create(3)); + + // re-introducing the column should not let the data re-appear + table.updateSchema().addColumn("data", Types.StringType.get()).commit(); + + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .as(Encoders.bean(SimpleRecord.class)) + .collectAsList()) + .containsExactly( + new SimpleRecord(1, null), new SimpleRecord(2, null), new SimpleRecord(3, null)); + } + + @Test + public void testWritingToBranchAfterSchemaChange() throws IOException { + String tableLocation = temp.newFolder("iceberg-table").toString(); + + HadoopTables tables = new HadoopTables(CONF); + PartitionSpec spec = PartitionSpec.unpartitioned(); + Table table = tables.create(SCHEMA, spec, properties, tableLocation); + + // produce the first snapshot + List firstBatchRecords = + Lists.newArrayList( + new SimpleRecord(1, "a"), new SimpleRecord(2, "b"), new SimpleRecord(3, "c")); + Dataset firstDf = spark.createDataFrame(firstBatchRecords, SimpleRecord.class); + firstDf.select("id", "data").write().format("iceberg").mode("append").save(tableLocation); + + table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); + + Dataset branchSnapshotResult = spark.read().format("iceberg").option("branch", "branch").load(tableLocation); - List deletedColumnBranchSnapshotRecords = - deletedColumnBranchSnapshotResult - .orderBy("id") - .as(Encoders.bean(SimpleRecord.class)) - .collectAsList(); + List branchSnapshotRecords = + branchSnapshotResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(firstBatchRecords); Assert.assertEquals( - "Current snapshot rows should match", expectedRecords, deletedColumnBranchSnapshotRecords); + "Current snapshot rows should match", expectedRecords, branchSnapshotRecords); + + // Deleting and add a new column of the same type to indicate schema change + table.updateSchema().deleteColumn("data").addColumn("zip", Types.IntegerType.get()).commit(); + + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .orderBy("id") + .collectAsList()) + .containsExactly( + RowFactory.create(1, null), RowFactory.create(2, null), RowFactory.create(3, null)); + + // writing new records into the branch should work with the new column + List records = + Lists.newArrayList( + RowFactory.create(4, 12345), RowFactory.create(5, 54321), RowFactory.create(6, 67890)); + + Dataset dataFrame = + spark.createDataFrame( + records, + SparkSchemaUtil.convert( + new Schema( + optional(1, "id", Types.IntegerType.get()), + optional(2, "zip", Types.IntegerType.get())))); + dataFrame + .select("id", "zip") + .write() + .format("iceberg") + .option("branch", "branch") + .mode("append") + .save(tableLocation); + + Assertions.assertThat( + spark + .read() + .format("iceberg") + .option("branch", "branch") + .load(tableLocation) + .collectAsList()) + .hasSize(6) + .contains( + RowFactory.create(1, null), RowFactory.create(2, null), RowFactory.create(3, null)) + .containsAll(records); } @Test From cbd33a6f626161bd1a4f0894a9dd4d96aed1a35b Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Tue, 5 Dec 2023 04:08:48 -0800 Subject: [PATCH 81/94] Flink: fix flaky test that might fail due to classloader check (#9216) --- .../source/TestIcebergSourceWithWatermarkExtractor.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 0bb2eb7766e9..aa4b9cd79e55 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -40,6 +40,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; @@ -92,7 +93,10 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(new Configuration())) + .setConfiguration( + reporter.addToConfiguration( + // disable classloader check as Avro may cache class in the serializers. + new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) .withHaLeadershipControl() .build()); From b4c050bc9c425b7b614058f420ba79b99db5d75f Mon Sep 17 00:00:00 2001 From: Li Han Date: Tue, 5 Dec 2023 20:30:25 +0800 Subject: [PATCH 82/94] Aliyun: Switch iceberg-aliyun's tests to JUnit5 (#9122) --- .../aliyun/TestAliyunClientFactories.java | 43 +++++------ .../apache/iceberg/aliyun/TestUtility.java | 27 +++---- ...STestRule.java => AliyunOSSExtension.java} | 27 +++---- .../iceberg/aliyun/oss/AliyunOSSTestBase.java | 23 +++--- ...Rule.java => OSSIntegrationExtension.java} | 4 +- .../iceberg/aliyun/oss/TestOSSFileIO.java | 75 +++++++++++-------- .../iceberg/aliyun/oss/TestOSSInputFile.java | 20 +++-- .../aliyun/oss/TestOSSInputStream.java | 23 +++--- .../iceberg/aliyun/oss/TestOSSOutputFile.java | 38 +++++----- .../aliyun/oss/TestOSSOutputStream.java | 30 ++++---- .../apache/iceberg/aliyun/oss/TestOSSURI.java | 31 ++++---- ...kRule.java => AliyunOSSMockExtension.java} | 10 +-- .../aliyun/oss/mock/TestLocalAliyunOSS.java | 59 ++++++++------- build.gradle | 3 + 14 files changed, 214 insertions(+), 199 deletions(-) rename aliyun/src/test/java/org/apache/iceberg/aliyun/oss/{AliyunOSSTestRule.java => AliyunOSSExtension.java} (83%) rename aliyun/src/test/java/org/apache/iceberg/aliyun/oss/{OSSIntegrationTestRule.java => OSSIntegrationExtension.java} (96%) rename aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/{AliyunOSSMockRule.java => AliyunOSSMockExtension.java} (92%) diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java index fa071e86051f..03df4af70b33 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java @@ -22,43 +22,44 @@ import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; public class TestAliyunClientFactories { @Test public void testLoadDefault() { - Assert.assertEquals( - "Default client should be singleton", - AliyunClientFactories.defaultFactory(), - AliyunClientFactories.defaultFactory()); + Assertions.assertThat(AliyunClientFactories.defaultFactory()) + .as("Default client should be singleton") + .isEqualTo(AliyunClientFactories.defaultFactory()); AliyunClientFactory defaultFactory = AliyunClientFactories.from(Maps.newHashMap()); - Assert.assertTrue( - "Should load default when factory impl not configured", - defaultFactory instanceof AliyunClientFactories.DefaultAliyunClientFactory); - Assert.assertNull( - "Should have no Aliyun properties set", defaultFactory.aliyunProperties().accessKeyId()); + Assertions.assertThat(defaultFactory) + .as("Should load default when factory impl not configured") + .isInstanceOf(AliyunClientFactories.DefaultAliyunClientFactory.class); + + Assertions.assertThat(defaultFactory.aliyunProperties().accessKeyId()) + .as("Should have no Aliyun properties set") + .isNull(); AliyunClientFactory defaultFactoryWithConfig = AliyunClientFactories.from(ImmutableMap.of(AliyunProperties.CLIENT_ACCESS_KEY_ID, "key")); - Assert.assertTrue( - "Should load default when factory impl not configured", - defaultFactoryWithConfig instanceof AliyunClientFactories.DefaultAliyunClientFactory); - Assert.assertEquals( - "Should have access key set", - "key", - defaultFactoryWithConfig.aliyunProperties().accessKeyId()); + Assertions.assertThat(defaultFactoryWithConfig) + .as("Should load default when factory impl not configured") + .isInstanceOf(AliyunClientFactories.DefaultAliyunClientFactory.class); + + Assertions.assertThat(defaultFactoryWithConfig.aliyunProperties().accessKeyId()) + .as("Should have access key set") + .isEqualTo("key"); } @Test public void testLoadCustom() { Map properties = Maps.newHashMap(); properties.put(AliyunProperties.CLIENT_FACTORY, CustomFactory.class.getName()); - Assert.assertTrue( - "Should load custom class", - AliyunClientFactories.from(properties) instanceof CustomFactory); + Assertions.assertThat(AliyunClientFactories.from(properties)) + .as("Should load custom class") + .isInstanceOf(CustomFactory.class); } public static class CustomFactory implements AliyunClientFactory { diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java index ac87a82fd7e0..072886f6b86b 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestUtility.java @@ -18,9 +18,9 @@ */ package org.apache.iceberg.aliyun; -import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule; +import org.apache.iceberg.aliyun.oss.AliyunOSSExtension; import org.apache.iceberg.aliyun.oss.OSSURI; -import org.apache.iceberg.aliyun.oss.mock.AliyunOSSMockRule; +import org.apache.iceberg.aliyun.oss.mock.AliyunOSSMockExtension; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; @@ -41,33 +41,34 @@ public class TestUtility { private TestUtility() {} - public static AliyunOSSTestRule initialize() { - AliyunOSSTestRule testRule; + public static AliyunOSSExtension initialize() { + AliyunOSSExtension extension; String implClass = System.getenv(ALIYUN_TEST_OSS_RULE_CLASS); if (!Strings.isNullOrEmpty(implClass)) { - LOG.info("The initializing AliyunOSSTestRule implementation is: {}", implClass); + LOG.info("The initializing AliyunOSSExtension implementation is: {}", implClass); try { - DynConstructors.Ctor ctor = - DynConstructors.builder(AliyunOSSTestRule.class).impl(implClass).buildChecked(); - testRule = ctor.newInstance(); + DynConstructors.Ctor ctor = + DynConstructors.builder(AliyunOSSExtension.class).impl(implClass).buildChecked(); + extension = ctor.newInstance(); } catch (NoSuchMethodException e) { throw new IllegalArgumentException( String.format( - "Cannot initialize AliyunOSSTestRule, missing no-arg constructor: %s", implClass), + "Cannot initialize AliyunOSSExtension, missing no-arg constructor: %s", implClass), e); } catch (ClassCastException e) { throw new IllegalArgumentException( String.format( - "Cannot initialize AliyunOSSTestRule, %s does not implement it.", implClass), + "Cannot initialize AliyunOSSExtension, %s does not implement it.", implClass), e); } } else { - LOG.info("Initializing AliyunOSSTestRule implementation with default AliyunOSSMockRule"); - testRule = AliyunOSSMockRule.builder().silent().build(); + LOG.info( + "Initializing AliyunOSSExtension implementation with default AliyunOSSMockExtension"); + extension = AliyunOSSMockExtension.builder().silent().build(); } - return testRule; + return extension; } public static String accessKeyId() { diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestRule.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSExtension.java similarity index 83% rename from aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestRule.java rename to aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSExtension.java index b9afa952aaa3..35b9c3e56f56 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestRule.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSExtension.java @@ -20,9 +20,9 @@ import com.aliyun.oss.OSS; import java.util.UUID; -import org.junit.rules.TestRule; -import org.junit.runner.Description; -import org.junit.runners.model.Statement; +import org.junit.jupiter.api.extension.AfterAllCallback; +import org.junit.jupiter.api.extension.BeforeAllCallback; +import org.junit.jupiter.api.extension.ExtensionContext; /** * API for test Aliyun Object Storage Service (OSS) which is either local mock http server or remote @@ -30,7 +30,7 @@ * *

This API includes start,stop OSS service, create OSS client, setup bucket and teardown bucket. */ -public interface AliyunOSSTestRule extends TestRule { +public interface AliyunOSSExtension extends BeforeAllCallback, AfterAllCallback { UUID RANDOM_UUID = java.util.UUID.randomUUID(); /** Returns a specific bucket name for testing purpose. */ @@ -39,18 +39,13 @@ default String testBucketName() { } @Override - default Statement apply(Statement base, Description description) { - return new Statement() { - @Override - public void evaluate() throws Throwable { - start(); - try { - base.evaluate(); - } finally { - stop(); - } - } - }; + default void afterAll(ExtensionContext context) throws Exception { + stop(); + } + + @Override + default void beforeAll(ExtensionContext context) throws Exception { + start(); } /** diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestBase.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestBase.java index 8b42cfe9bd18..f34bac2e33e5 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestBase.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/AliyunOSSTestBase.java @@ -21,25 +21,26 @@ import com.aliyun.oss.OSS; import org.apache.iceberg.aliyun.TestUtility; import org.apache.iceberg.util.SerializableSupplier; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.RegisterExtension; public abstract class AliyunOSSTestBase { - @ClassRule public static final AliyunOSSTestRule OSS_TEST_RULE = TestUtility.initialize(); + @RegisterExtension + private static final AliyunOSSExtension OSS_TEST_EXTENSION = TestUtility.initialize(); - private final SerializableSupplier ossClient = OSS_TEST_RULE::createOSSClient; - private final String bucketName = OSS_TEST_RULE.testBucketName(); - private final String keyPrefix = OSS_TEST_RULE.keyPrefix(); + private final SerializableSupplier ossClient = OSS_TEST_EXTENSION::createOSSClient; + private final String bucketName = OSS_TEST_EXTENSION.testBucketName(); + private final String keyPrefix = OSS_TEST_EXTENSION.keyPrefix(); - @Before + @BeforeEach public void before() { - OSS_TEST_RULE.setUpBucket(bucketName); + OSS_TEST_EXTENSION.setUpBucket(bucketName); } - @After + @AfterEach public void after() { - OSS_TEST_RULE.tearDownBucket(bucketName); + OSS_TEST_EXTENSION.tearDownBucket(bucketName); } protected String location(String key) { diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/OSSIntegrationTestRule.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/OSSIntegrationExtension.java similarity index 96% rename from aliyun/src/test/java/org/apache/iceberg/aliyun/oss/OSSIntegrationTestRule.java rename to aliyun/src/test/java/org/apache/iceberg/aliyun/oss/OSSIntegrationExtension.java index 21e427385a9d..b5ecfc579283 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/OSSIntegrationTestRule.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/OSSIntegrationExtension.java @@ -26,7 +26,7 @@ import org.apache.iceberg.aliyun.TestUtility; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -public class OSSIntegrationTestRule implements AliyunOSSTestRule { +public class OSSIntegrationExtension implements AliyunOSSExtension { // Aliyun access key pair. private String accessKeyId; private String accessKeySecret; @@ -106,7 +106,7 @@ public void tearDownBucket(String bucket) { private OSS ossClient() { if (lazyClient == null) { - synchronized (OSSIntegrationTestRule.class) { + synchronized (OSSIntegrationExtension.class) { if (lazyClient == null) { lazyClient = createOSSClient(); } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java index febbf3fe33b5..1cc8f45467e2 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java @@ -39,10 +39,10 @@ import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.apache.iceberg.util.SerializableSupplier; import org.apache.iceberg.util.SerializationUtil; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestOSSFileIO extends AliyunOSSTestBase { private static final String OSS_IMPL_CLASS = OSSFileIO.class.getName(); @@ -51,12 +51,12 @@ public class TestOSSFileIO extends AliyunOSSTestBase { private FileIO fileIO; - @Before + @BeforeEach public void beforeFile() { fileIO = new OSSFileIO(ossClient()); } - @After + @AfterEach public void afterFile() { if (fileIO != null) { fileIO.close(); @@ -73,28 +73,33 @@ public void testOutputFile() throws IOException { writeOSSData(out, data); OSSURI uri = new OSSURI(location); - Assert.assertTrue( - "OSS file should exist", ossClient().get().doesObjectExist(uri.bucket(), uri.key())); - Assert.assertEquals("Should have expected location", location, out.location()); - Assert.assertEquals("Should have expected length", dataSize, ossDataLength(uri)); - Assert.assertArrayEquals("Should have expected content", data, ossDataContent(uri, dataSize)); + Assertions.assertThat(ossClient().get().doesObjectExist(uri.bucket(), uri.key())) + .as("OSS file should exist") + .isTrue(); + Assertions.assertThat(out.location()).as("Should have expected location").isEqualTo(location); + Assertions.assertThat(ossDataLength(uri)).as("Should have expected length").isEqualTo(dataSize); + Assertions.assertThat(ossDataContent(uri, dataSize)) + .as("Should have expected content") + .isEqualTo(data); } @Test public void testInputFile() throws IOException { String location = randomLocation(); InputFile in = fileIO().newInputFile(location); - Assert.assertFalse("OSS file should not exist", in.exists()); + Assertions.assertThat(in.exists()).as("OSS file should not exist").isFalse(); int dataSize = 1024 * 10; byte[] data = randomData(dataSize); OutputFile out = fileIO().newOutputFile(location); writeOSSData(out, data); - Assert.assertTrue("OSS file should exist", in.exists()); - Assert.assertEquals("Should have expected location", location, in.location()); - Assert.assertEquals("Should have expected length", dataSize, in.getLength()); - Assert.assertArrayEquals("Should have expected content", data, inFileContent(in, dataSize)); + Assertions.assertThat(in.exists()).as("OSS file should exist").isTrue(); + Assertions.assertThat(in.location()).as("Should have expected location").isEqualTo(location); + Assertions.assertThat(in.getLength()).as("Should have expected length").isEqualTo(dataSize); + Assertions.assertThat(inFileContent(in, dataSize)) + .as("Should have expected content") + .isEqualTo(data); } @Test @@ -106,20 +111,24 @@ public void testDeleteFile() throws IOException { writeOSSData(out, data); InputFile in = fileIO().newInputFile(location); - Assert.assertTrue("OSS file should exist", in.exists()); + Assertions.assertThat(in.exists()).as("OSS file should exist").isTrue(); + fileIO().deleteFile(in); - Assert.assertFalse("OSS file should not exist", fileIO().newInputFile(location).exists()); + Assertions.assertThat(fileIO().newInputFile(location).exists()) + .as("OSS file should not exist") + .isFalse(); } @Test public void testLoadFileIO() { FileIO file = CatalogUtil.loadFileIO(OSS_IMPL_CLASS, ImmutableMap.of(), conf); - Assert.assertTrue("Should be OSSFileIO", file instanceof OSSFileIO); + Assertions.assertThat(file).as("Should be OSSFileIO").isInstanceOf(OSSFileIO.class); byte[] data = SerializationUtil.serializeToBytes(file); FileIO expectedFileIO = SerializationUtil.deserializeFromBytes(data); - Assert.assertTrue( - "The deserialized FileIO should be OSSFileIO", expectedFileIO instanceof OSSFileIO); + Assertions.assertThat(expectedFileIO) + .as("The deserialized FileIO should be OSSFileIO") + .isInstanceOf(OSSFileIO.class); } @Test @@ -134,19 +143,21 @@ public void serializeClient() throws URISyntaxException { SerializableSupplier post = SerializationUtil.deserializeFromBytes(data); OSS client = post.get(); - Assert.assertTrue("Should be instance of oss client", client instanceof OSSClient); + Assertions.assertThat(client) + .as("Should be instance of oss client") + .isInstanceOf(OSSClient.class); OSSClient oss = (OSSClient) client; - Assert.assertEquals( - "Should have expected endpoint", new URI("http://" + endpoint), oss.getEndpoint()); - Assert.assertEquals( - "Should have expected access key", - accessKeyId, - oss.getCredentialsProvider().getCredentials().getAccessKeyId()); - Assert.assertEquals( - "Should have expected secret key", - accessSecret, - oss.getCredentialsProvider().getCredentials().getSecretAccessKey()); + Assertions.assertThat(oss.getEndpoint()) + .as("Should have expected endpoint") + .isEqualTo(new URI("http://" + endpoint)); + + Assertions.assertThat(oss.getCredentialsProvider().getCredentials().getAccessKeyId()) + .as("Should have expected access key") + .isEqualTo(accessKeyId); + Assertions.assertThat(oss.getCredentialsProvider().getCredentials().getSecretAccessKey()) + .as("Should have expected secret key") + .isEqualTo(accessSecret); } private FileIO fileIO() { diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputFile.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputFile.java index 86d2ef729584..8d7cf51ccc84 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputFile.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputFile.java @@ -37,8 +37,7 @@ import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestOSSInputFile extends AliyunOSSTestBase { private final OSS ossClient = ossClient().get(); @@ -75,7 +74,7 @@ public void testExists() { InputFile inputFile = new OSSInputFile(ossMock, uri, aliyunProperties, MetricsContext.nullMetrics()); - Assert.assertFalse("OSS file should not exist", inputFile.exists()); + Assertions.assertThat(inputFile.exists()).as("OSS file should not exist").isFalse(); verify(ossMock, times(1)).getSimplifiedObjectMeta(uri.bucket(), uri.key()); reset(ossMock); @@ -83,7 +82,7 @@ public void testExists() { byte[] data = randomData(dataSize); writeOSSData(uri, data); - Assert.assertTrue("OSS file should exist", inputFile.exists()); + Assertions.assertThat(inputFile.exists()).as("OSS file should exist").isTrue(); inputFile.exists(); verify(ossMock, times(1)).getSimplifiedObjectMeta(uri.bucket(), uri.key()); reset(ossMock); @@ -109,14 +108,17 @@ public void testGetLength() { private void readAndVerify(OSSURI uri, byte[] data) throws IOException { InputFile inputFile = new OSSInputFile(ossClient().get(), uri, aliyunProperties, MetricsContext.nullMetrics()); - Assert.assertTrue("OSS file should exist", inputFile.exists()); - Assert.assertEquals("Should have expected file length", data.length, inputFile.getLength()); + Assertions.assertThat(inputFile.exists()).as("OSS file should exist").isTrue(); + Assertions.assertThat(inputFile.getLength()) + .as("Should have expected file length") + .isEqualTo(data.length); byte[] actual = new byte[data.length]; try (SeekableInputStream in = inputFile.newStream()) { ByteStreams.readFully(in, actual); } - Assert.assertArrayEquals("Should have same object content", data, actual); + + Assertions.assertThat(actual).as("Should have same object content").isEqualTo(data); } private void verifyLength(OSS ossClientMock, OSSURI uri, byte[] data, boolean isCache) { @@ -130,7 +132,9 @@ private void verifyLength(OSS ossClientMock, OSSURI uri, byte[] data, boolean is new OSSInputFile(ossClientMock, uri, aliyunProperties, MetricsContext.nullMetrics()); } inputFile.getLength(); - Assert.assertEquals("Should have expected file length", data.length, inputFile.getLength()); + Assertions.assertThat(inputFile.getLength()) + .as("Should have expected file length") + .isEqualTo(data.length); } private OSSURI randomURI() { diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputStream.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputStream.java index 6798137c47b7..ccbfa39c9959 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputStream.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputStream.java @@ -18,9 +18,6 @@ */ package org.apache.iceberg.aliyun.oss; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; - import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.Arrays; @@ -29,7 +26,7 @@ import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestOSSInputStream extends AliyunOSSTestBase { private final Random random = ThreadLocalRandom.current(); @@ -72,7 +69,7 @@ private void readAndCheck( SeekableInputStream in, long rangeStart, int size, byte[] original, boolean buffered) throws IOException { in.seek(rangeStart); - assertEquals("Should have the correct position", rangeStart, in.getPos()); + Assertions.assertThat(in.getPos()).as("Should have the correct position").isEqualTo(rangeStart); long rangeEnd = rangeStart + size; byte[] actual = new byte[size]; @@ -86,12 +83,11 @@ private void readAndCheck( } } - assertEquals("Should have the correct position", rangeEnd, in.getPos()); + Assertions.assertThat(in.getPos()).as("Should have the correct position").isEqualTo(rangeEnd); - assertArrayEquals( - "Should have expected range data", - Arrays.copyOfRange(original, (int) rangeStart, (int) rangeEnd), - actual); + Assertions.assertThat(actual) + .as("Should have expected range data") + .isEqualTo(Arrays.copyOfRange(original, (int) rangeStart, (int) rangeEnd)); } @Test @@ -115,10 +111,9 @@ public void testSeek() throws Exception { in.seek(expected.length / 2); byte[] actual = new byte[expected.length / 2]; ByteStreams.readFully(in, actual); - assertArrayEquals( - "Should have expected seeking stream", - Arrays.copyOfRange(expected, expected.length / 2, expected.length), - actual); + Assertions.assertThat(actual) + .as("Should have expected seeking stream") + .isEqualTo(Arrays.copyOfRange(expected, expected.length / 2, expected.length)); } } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputFile.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputFile.java index bcb36a033f00..75f095112b02 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputFile.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputFile.java @@ -33,8 +33,7 @@ import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestOSSOutputFile extends AliyunOSSTestBase { @@ -54,11 +53,14 @@ public void testWriteFile() throws IOException { ByteStreams.copy(is, os); } - Assert.assertTrue("OSS file should exist", ossClient.doesObjectExist(uri.bucket(), uri.key())); - Assert.assertEquals("Object length should match", ossDataLength(uri), dataSize); + Assertions.assertThat(ossClient.doesObjectExist(uri.bucket(), uri.key())) + .as("OSS file should exist") + .isTrue(); + + Assertions.assertThat(ossDataLength(uri)).as("Object length should match").isEqualTo(dataSize); byte[] actual = ossDataContent(uri, dataSize); - Assert.assertArrayEquals("Object content should match", data, actual); + Assertions.assertThat(actual).as("Object content should match").isEqualTo(data); } @Test @@ -100,14 +102,12 @@ public void testCreateOrOverwrite() throws IOException { InputStream is = new ByteArrayInputStream(expect)) { ByteStreams.copy(is, os); } - - Assert.assertEquals( - String.format("Should overwrite object length from %d to %d", dataSize, expectSize), - expectSize, - ossDataLength(uri)); + Assertions.assertThat(ossDataLength(uri)) + .as(String.format("Should overwrite object length from %d to %d", dataSize, expectSize)) + .isEqualTo(expectSize); byte[] actual = ossDataContent(uri, expectSize); - Assert.assertArrayEquals("Should overwrite object content", expect, actual); + Assertions.assertThat(actual).as("Should overwrite object content").isEqualTo(expect); } @Test @@ -115,7 +115,7 @@ public void testLocation() { OSSURI uri = randomURI(); OutputFile out = new OSSOutputFile(ossClient, uri, aliyunProperties, MetricsContext.nullMetrics()); - Assert.assertEquals("Location should match", uri.location(), out.location()); + Assertions.assertThat(out.location()).as("Location should match").isEqualTo(uri.location()); } @Test @@ -131,16 +131,20 @@ public void testToInputFile() throws IOException { } InputFile in = out.toInputFile(); - Assert.assertTrue("Should be an instance of OSSInputFile", in instanceof OSSInputFile); - Assert.assertTrue("OSS file should exist", in.exists()); - Assert.assertEquals("Should have expected location", out.location(), in.location()); - Assert.assertEquals("Should have expected length", dataSize, in.getLength()); + Assertions.assertThat(in) + .as("Should be an instance of OSSInputFile") + .isInstanceOf(OSSInputFile.class); + Assertions.assertThat(in.exists()).as("OSS file should exist").isTrue(); + Assertions.assertThat(in.location()) + .as("Should have expected location") + .isEqualTo(out.location()); + Assertions.assertThat(in.getLength()).as("Should have expected length").isEqualTo(dataSize); byte[] actual = new byte[dataSize]; try (InputStream as = in.newStream()) { ByteStreams.readFully(as, actual); } - Assert.assertArrayEquals("Should have expected content", data, actual); + Assertions.assertThat(actual).as("Should have expected content").isEqualTo(data); } private OSSURI randomURI() { diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java index 9fa7a648f8dc..fadad545d07f 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputStream.java @@ -38,8 +38,8 @@ import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.junit.Assert; -import org.junit.Test; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,30 +88,30 @@ private void writeAndVerify(OSS mock, OSSURI uri, byte[] data, boolean arrayWrit new OSSOutputStream(mock, uri, props, MetricsContext.nullMetrics())) { if (arrayWrite) { out.write(data); - Assert.assertEquals("OSSOutputStream position", data.length, out.getPos()); + Assertions.assertThat(out.getPos()).as("OSSOutputStream position").isEqualTo(data.length); } else { for (int i = 0; i < data.length; i++) { out.write(data[i]); - Assert.assertEquals("OSSOutputStream position", i + 1, out.getPos()); + Assertions.assertThat(out.getPos()).as("OSSOutputStream position").isEqualTo(i + 1); } } } - Assert.assertTrue( - "OSS object should exist", ossClient.doesObjectExist(uri.bucket(), uri.key())); - Assert.assertEquals( - "Object length", - ossClient.getObject(uri.bucket(), uri.key()).getObjectMetadata().getContentLength(), - data.length); + Assertions.assertThat(ossClient.doesObjectExist(uri.bucket(), uri.key())) + .as("OSS object should exist") + .isTrue(); + Assertions.assertThat( + ossClient.getObject(uri.bucket(), uri.key()).getObjectMetadata().getContentLength()) + .as("Object length") + .isEqualTo(data.length); byte[] actual = ossDataContent(uri, data.length); - Assert.assertArrayEquals("Object content", data, actual); + Assertions.assertThat(actual).as("Object content").isEqualTo(data); // Verify all staging files are cleaned up. - Assert.assertEquals( - "Staging files should clean up", - 0, - Files.list(Paths.get(props.ossStagingDirectory())).count()); + Assertions.assertThat(Files.list(Paths.get(props.ossStagingDirectory())).count()) + .as("Staging files should clean up") + .isEqualTo(0); } private OSSURI randomURI() { diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java index 82692bfa7134..932dfe51c4f3 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java @@ -23,8 +23,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestOSSURI { @Test @@ -32,9 +31,9 @@ public void testUrlParsing() { String location = "oss://bucket/path/to/file"; OSSURI uri = new OSSURI(location); - Assert.assertEquals("bucket", uri.bucket()); - Assert.assertEquals("path/to/file", uri.key()); - Assert.assertEquals(location, uri.toString()); + Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); + Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); + Assertions.assertThat(uri.toString()).isEqualTo(location); } @Test @@ -42,9 +41,9 @@ public void testEncodedString() { String location = "oss://bucket/path%20to%20file"; OSSURI uri = new OSSURI(location); - Assert.assertEquals("bucket", uri.bucket()); - Assert.assertEquals("path%20to%20file", uri.key()); - Assert.assertEquals(location, uri.toString()); + Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); + Assertions.assertThat(uri.key()).isEqualTo("path%20to%20file"); + Assertions.assertThat(uri.toString()).isEqualTo(location); } @Test @@ -93,9 +92,9 @@ public void testFragment() { String location = "oss://bucket/path/to/file#print"; OSSURI uri = new OSSURI(location); - Assert.assertEquals("bucket", uri.bucket()); - Assert.assertEquals("path/to/file", uri.key()); - Assert.assertEquals(location, uri.toString()); + Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); + Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); + Assertions.assertThat(uri.toString()).isEqualTo(location); } @Test @@ -103,17 +102,17 @@ public void testQueryAndFragment() { String location = "oss://bucket/path/to/file?query=foo#bar"; OSSURI uri = new OSSURI(location); - Assert.assertEquals("bucket", uri.bucket()); - Assert.assertEquals("path/to/file", uri.key()); - Assert.assertEquals(location, uri.toString()); + Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); + Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); + Assertions.assertThat(uri.toString()).isEqualTo(location); } @Test public void testValidSchemes() { for (String scheme : Lists.newArrayList("https", "oss")) { OSSURI uri = new OSSURI(scheme + "://bucket/path/to/file"); - Assert.assertEquals("bucket", uri.bucket()); - Assert.assertEquals("path/to/file", uri.key()); + Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); + Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); } } } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java similarity index 92% rename from aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java rename to aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java index d23d4f1d83f3..5a9b06090392 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockRule.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/AliyunOSSMockExtension.java @@ -25,18 +25,18 @@ import java.io.UncheckedIOException; import java.nio.file.Files; import java.util.Map; -import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule; +import org.apache.iceberg.aliyun.oss.AliyunOSSExtension; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -public class AliyunOSSMockRule implements AliyunOSSTestRule { +public class AliyunOSSMockExtension implements AliyunOSSExtension { private final Map properties; private AliyunOSSMockApp ossMockApp; - private AliyunOSSMockRule(Map properties) { + private AliyunOSSMockExtension(Map properties) { this.properties = properties; } @@ -108,7 +108,7 @@ public Builder silent() { return this; } - public AliyunOSSTestRule build() { + public AliyunOSSExtension build() { String rootDir = (String) props.get(AliyunOSSMockApp.PROP_ROOT_DIR); if (Strings.isNullOrEmpty(rootDir)) { File dir = @@ -121,7 +121,7 @@ public AliyunOSSTestRule build() { File root = new File(rootDir); root.deleteOnExit(); root.mkdir(); - return new AliyunOSSMockRule(props); + return new AliyunOSSMockExtension(props); } } } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java index ac9088256873..5a47708fbd41 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java @@ -31,23 +31,23 @@ import java.util.Random; import java.util.UUID; import org.apache.iceberg.aliyun.TestUtility; -import org.apache.iceberg.aliyun.oss.AliyunOSSTestRule; +import org.apache.iceberg.aliyun.oss.AliyunOSSExtension; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.assertj.core.api.Assertions; +import org.assertj.core.api.Assumptions; import org.assertj.core.api.InstanceOfAssertFactories; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestLocalAliyunOSS { - @ClassRule public static final AliyunOSSTestRule OSS_TEST_RULE = TestUtility.initialize(); + @RegisterExtension + private static final AliyunOSSExtension OSS_TEST_EXTENSION = TestUtility.initialize(); - private final OSS oss = OSS_TEST_RULE.createOSSClient(); - private final String bucketName = OSS_TEST_RULE.testBucketName(); + private final OSS oss = OSS_TEST_EXTENSION.createOSSClient(); + private final String bucketName = OSS_TEST_EXTENSION.testBucketName(); private final Random random = new Random(1); private static void assertThrows(Runnable runnable, String expectedErrorCode) { @@ -58,37 +58,38 @@ private static void assertThrows(Runnable runnable, String expectedErrorCode) { .isEqualTo(expectedErrorCode); } - @Before + @BeforeEach public void before() { - OSS_TEST_RULE.setUpBucket(bucketName); + OSS_TEST_EXTENSION.setUpBucket(bucketName); } - @After + @AfterEach public void after() { - OSS_TEST_RULE.tearDownBucket(bucketName); + OSS_TEST_EXTENSION.tearDownBucket(bucketName); } @Test public void testBuckets() { - Assume.assumeTrue( - "Aliyun integration test cannot delete existing bucket from test environment.", - OSS_TEST_RULE.getClass() == AliyunOSSMockRule.class); + Assumptions.assumeThat(OSS_TEST_EXTENSION.getClass()) + .as("Aliyun integration test cannot delete existing bucket from test environment.") + .isEqualTo(AliyunOSSMockExtension.class); + + Assertions.assertThat(doesBucketExist(bucketName)).isTrue(); - Assert.assertTrue(doesBucketExist(bucketName)); assertThrows(() -> oss.createBucket(bucketName), OSSErrorCode.BUCKET_ALREADY_EXISTS); oss.deleteBucket(bucketName); - Assert.assertFalse(doesBucketExist(bucketName)); + Assertions.assertThat(doesBucketExist(bucketName)).isFalse(); oss.createBucket(bucketName); - Assert.assertTrue(doesBucketExist(bucketName)); + Assertions.assertThat(doesBucketExist(bucketName)).isTrue(); } @Test public void testDeleteBucket() { - Assume.assumeTrue( - "Aliyun integration test cannot delete existing bucket from test environment.", - OSS_TEST_RULE.getClass() == AliyunOSSMockRule.class); + Assumptions.assumeThat(OSS_TEST_EXTENSION.getClass()) + .as("Aliyun integration test cannot delete existing bucket from test environment.") + .isEqualTo(AliyunOSSMockExtension.class); String bucketNotExist = String.format("bucket-not-existing-%s", UUID.randomUUID()); assertThrows(() -> oss.deleteBucket(bucketNotExist), OSSErrorCode.NO_SUCH_BUCKET); @@ -107,7 +108,7 @@ public void testDeleteBucket() { oss.deleteObject(bucketName, "object2"); oss.deleteBucket(bucketName); - Assert.assertFalse(doesBucketExist(bucketName)); + Assertions.assertThat(doesBucketExist(bucketName)).isFalse(); oss.createBucket(bucketName); } @@ -122,18 +123,18 @@ public void testPutObject() throws IOException { () -> oss.putObject(bucketNotExist, "object", wrap(bytes)), OSSErrorCode.NO_SUCH_BUCKET); PutObjectResult result = oss.putObject(bucketName, "object", wrap(bytes)); - Assert.assertEquals(AliyunOSSMockLocalStore.md5sum(wrap(bytes)), result.getETag()); + Assertions.assertThat(result.getETag()).isEqualTo(AliyunOSSMockLocalStore.md5sum(wrap(bytes))); } @Test public void testDoesObjectExist() { - Assert.assertFalse(oss.doesObjectExist(bucketName, "key")); + Assertions.assertThat(oss.doesObjectExist(bucketName, "key")).isFalse(); byte[] bytes = new byte[4 * 1024]; random.nextBytes(bytes); oss.putObject(bucketName, "key", wrap(bytes)); - Assert.assertTrue(oss.doesObjectExist(bucketName, "key")); + Assertions.assertThat(oss.doesObjectExist(bucketName, "key")).isTrue(); oss.deleteObject(bucketName, "key"); } @@ -153,7 +154,7 @@ public void testGetObject() throws IOException { try (InputStream is = oss.getObject(bucketName, "key").getObjectContent()) { ByteStreams.readFully(is, actual); } - Assert.assertArrayEquals(bytes, actual); + Assertions.assertThat(actual).isEqualTo(bytes); oss.deleteObject(bucketName, "key"); } @@ -229,7 +230,7 @@ private void testRange(byte[] bytes, int start, int end) throws IOException { try (InputStream is = oss.getObject(getObjectRequest).getObjectContent()) { ByteStreams.readFully(is, actual); } - Assert.assertArrayEquals(testBytes, actual); + Assertions.assertThat(actual).isEqualTo(testBytes); } private InputStream wrap(byte[] data) { diff --git a/build.gradle b/build.gradle index 679ffc6f2fc8..94996a41a648 100644 --- a/build.gradle +++ b/build.gradle @@ -413,6 +413,9 @@ project(':iceberg-data') { } project(':iceberg-aliyun') { + test { + useJUnitPlatform() + } dependencies { implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') api project(':iceberg-api') From f19643a93f5dac99bbdbc9881ef19c89d7bcd3eb Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 5 Dec 2023 17:03:47 +0100 Subject: [PATCH 83/94] Core: Add View support for REST catalog (#7913) --- .../apache/iceberg/catalog/ViewCatalog.java | 1 + .../iceberg/catalog/ViewSessionCatalog.java | 120 ++++ .../org/apache/iceberg/UpdateRequirement.java | 15 + .../catalog/BaseViewSessionCatalog.java | 92 +++ .../apache/iceberg/rest/CatalogHandlers.java | 137 ++++ .../apache/iceberg/rest/ErrorHandlers.java | 52 ++ .../org/apache/iceberg/rest/RESTCatalog.java | 43 +- .../apache/iceberg/rest/RESTSerializers.java | 51 +- .../iceberg/rest/RESTSessionCatalog.java | 260 +++++++- .../iceberg/rest/RESTViewOperations.java | 83 +++ .../apache/iceberg/rest/ResourcePaths.java | 18 + .../rest/requests/CreateViewRequest.java | 45 ++ .../requests/CreateViewRequestParser.java | 99 +++ .../rest/requests/RenameTableRequest.java | 2 +- .../rest/responses/LoadViewResponse.java | 38 ++ .../responses/LoadViewResponseParser.java | 85 +++ .../org/apache/iceberg/view/ViewMetadata.java | 38 +- .../iceberg/view/ViewMetadataParser.java | 2 +- .../iceberg/view/ViewVersionParser.java | 2 +- .../iceberg/view/ViewVersionReplace.java | 2 +- .../iceberg/rest/RESTCatalogAdapter.java | 95 ++- .../iceberg/rest/TestRESTViewCatalog.java | 166 +++++ .../iceberg/rest/TestResourcePaths.java | 47 ++ .../requests/TestCreateViewRequestParser.java | 130 ++++ .../responses/TestLoadViewResponseParser.java | 260 ++++++++ .../apache/iceberg/view/TestViewMetadata.java | 23 +- .../apache/iceberg/view/ViewCatalogTests.java | 144 ++++- open-api/rest-catalog-open-api.py | 111 ++++ open-api/rest-catalog-open-api.yaml | 595 +++++++++++++++++- 29 files changed, 2722 insertions(+), 34 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/catalog/ViewSessionCatalog.java create mode 100644 core/src/main/java/org/apache/iceberg/catalog/BaseViewSessionCatalog.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/requests/CreateViewRequest.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/requests/CreateViewRequestParser.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java create mode 100644 core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/requests/TestCreateViewRequestParser.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java diff --git a/api/src/main/java/org/apache/iceberg/catalog/ViewCatalog.java b/api/src/main/java/org/apache/iceberg/catalog/ViewCatalog.java index df118e25964f..ca470eec7171 100644 --- a/api/src/main/java/org/apache/iceberg/catalog/ViewCatalog.java +++ b/api/src/main/java/org/apache/iceberg/catalog/ViewCatalog.java @@ -92,6 +92,7 @@ default boolean viewExists(TableIdentifier identifier) { * @param to new view identifier * @throws NoSuchViewException if the "from" view does not exist * @throws AlreadyExistsException if the "to" view already exists + * @throws NoSuchNamespaceException if the "to" namespace doesn't exist */ void renameView(TableIdentifier from, TableIdentifier to); diff --git a/api/src/main/java/org/apache/iceberg/catalog/ViewSessionCatalog.java b/api/src/main/java/org/apache/iceberg/catalog/ViewSessionCatalog.java new file mode 100644 index 000000000000..106e20d3bce1 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/catalog/ViewSessionCatalog.java @@ -0,0 +1,120 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.catalog; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchViewException; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewBuilder; + +/** A session Catalog API for view create, drop, and load operations. */ +public interface ViewSessionCatalog { + + /** + * Return the name for this catalog. + * + * @return this catalog's name + */ + String name(); + + /** + * Return all the identifiers under this namespace. + * + * @param namespace a namespace + * @return a list of identifiers for views + * @throws NoSuchNamespaceException if the namespace is not found + */ + List listViews(SessionCatalog.SessionContext context, Namespace namespace); + + /** + * Load a view. + * + * @param identifier a view identifier + * @return instance of {@link View} implementation referred by the identifier + * @throws NoSuchViewException if the view does not exist + */ + View loadView(SessionCatalog.SessionContext context, TableIdentifier identifier); + + /** + * Check whether view exists. + * + * @param identifier a view identifier + * @return true if the view exists, false otherwise + */ + default boolean viewExists(SessionCatalog.SessionContext context, TableIdentifier identifier) { + try { + loadView(context, identifier); + return true; + } catch (NoSuchViewException e) { + return false; + } + } + + /** + * Instantiate a builder to create or replace a SQL view. + * + * @param identifier a view identifier + * @return a view builder + */ + ViewBuilder buildView(SessionCatalog.SessionContext context, TableIdentifier identifier); + + /** + * Drop a view. + * + * @param identifier a view identifier + * @return true if the view was dropped, false if the view did not exist + */ + boolean dropView(SessionCatalog.SessionContext context, TableIdentifier identifier); + + /** + * Rename a view. + * + * @param from identifier of the view to rename + * @param to new view identifier + * @throws NoSuchViewException if the "from" view does not exist + * @throws AlreadyExistsException if the "to" view already exists + * @throws NoSuchNamespaceException if the "to" namespace doesn't exist + */ + void renameView(SessionCatalog.SessionContext context, TableIdentifier from, TableIdentifier to); + + /** + * Invalidate cached view metadata from current catalog. + * + *

If the view is already loaded or cached, drop cached data. If the view does not exist or is + * not cached, do nothing. + * + * @param identifier a view identifier + */ + default void invalidateView(SessionCatalog.SessionContext context, TableIdentifier identifier) {} + + /** + * Initialize a view catalog given a custom name and a map of catalog properties. + * + *

A custom view catalog implementation must have a no-arg constructor. A compute engine like + * Spark or Flink will first initialize the catalog without any arguments, and then call this + * method to complete catalog initialization with properties passed into the engine. + * + * @param name a custom name for the catalog + * @param properties catalog properties + */ + void initialize(String name, Map properties); +} diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirement.java b/core/src/main/java/org/apache/iceberg/UpdateRequirement.java index 2645e79fa4c5..80ecf84efa45 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateRequirement.java +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirement.java @@ -19,12 +19,19 @@ package org.apache.iceberg; import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.view.ViewMetadata; /** Represents a requirement for a {@link MetadataUpdate} */ public interface UpdateRequirement { void validate(TableMetadata base); + default void validate(ViewMetadata base) { + throw new ValidationException( + "Cannot validate %s against a view", this.getClass().getSimpleName()); + } + class AssertTableDoesNotExist implements UpdateRequirement { public AssertTableDoesNotExist() {} @@ -55,6 +62,14 @@ public void validate(TableMetadata base) { "Requirement failed: UUID does not match: expected %s != %s", base.uuid(), uuid); } } + + @Override + public void validate(ViewMetadata base) { + if (!uuid.equalsIgnoreCase(base.uuid())) { + throw new CommitFailedException( + "Requirement failed: UUID does not match: expected %s != %s", base.uuid(), uuid); + } + } } class AssertRefSnapshotID implements UpdateRequirement { diff --git a/core/src/main/java/org/apache/iceberg/catalog/BaseViewSessionCatalog.java b/core/src/main/java/org/apache/iceberg/catalog/BaseViewSessionCatalog.java new file mode 100644 index 000000000000..10895e1de9e6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/catalog/BaseViewSessionCatalog.java @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.catalog; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewBuilder; + +public abstract class BaseViewSessionCatalog extends BaseSessionCatalog + implements ViewSessionCatalog { + + private final Cache catalogs = + Caffeine.newBuilder().expireAfterAccess(10, TimeUnit.MINUTES).build(); + + public ViewCatalog asViewCatalog(SessionContext context) { + return catalogs.get(context.sessionId(), id -> new AsViewCatalog(context)); + } + + public class AsViewCatalog implements ViewCatalog { + private final SessionContext context; + + private AsViewCatalog(SessionContext context) { + this.context = context; + } + + @Override + public String name() { + return BaseViewSessionCatalog.this.name(); + } + + @Override + public List listViews(Namespace namespace) { + return BaseViewSessionCatalog.this.listViews(context, namespace); + } + + @Override + public View loadView(TableIdentifier identifier) { + return BaseViewSessionCatalog.this.loadView(context, identifier); + } + + @Override + public boolean viewExists(TableIdentifier identifier) { + return BaseViewSessionCatalog.this.viewExists(context, identifier); + } + + @Override + public ViewBuilder buildView(TableIdentifier identifier) { + return BaseViewSessionCatalog.this.buildView(context, identifier); + } + + @Override + public boolean dropView(TableIdentifier identifier) { + return BaseViewSessionCatalog.this.dropView(context, identifier); + } + + @Override + public void renameView(TableIdentifier from, TableIdentifier to) { + BaseViewSessionCatalog.this.renameView(context, from, to); + } + + @Override + public void invalidateView(TableIdentifier identifier) { + BaseViewSessionCatalog.this.invalidateView(context, identifier); + } + + @Override + public void initialize(String name, Map properties) { + throw new UnsupportedOperationException( + this.getClass().getSimpleName() + " doesn't support initialization"); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java index 1e0ef660271b..e4e3c065fb55 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -45,26 +45,38 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.catalog.ViewCatalog; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.rest.requests.CreateNamespaceRequest; import org.apache.iceberg.rest.requests.CreateTableRequest; +import org.apache.iceberg.rest.requests.CreateViewRequest; import org.apache.iceberg.rest.requests.RegisterTableRequest; import org.apache.iceberg.rest.requests.RenameTableRequest; import org.apache.iceberg.rest.requests.UpdateNamespacePropertiesRequest; import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.responses.CreateNamespaceResponse; import org.apache.iceberg.rest.responses.GetNamespaceResponse; +import org.apache.iceberg.rest.responses.ImmutableLoadViewResponse; import org.apache.iceberg.rest.responses.ListNamespacesResponse; import org.apache.iceberg.rest.responses.ListTablesResponse; import org.apache.iceberg.rest.responses.LoadTableResponse; +import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse; import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.view.BaseView; +import org.apache.iceberg.view.SQLViewRepresentation; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewBuilder; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewOperations; +import org.apache.iceberg.view.ViewRepresentation; public class CatalogHandlers { private static final Schema EMPTY_SCHEMA = new Schema(); @@ -374,4 +386,129 @@ static TableMetadata commit(TableOperations ops, UpdateTableRequest request) { return ops.current(); } + + private static BaseView asBaseView(View view) { + Preconditions.checkState( + view instanceof BaseView, "Cannot wrap catalog that does not produce BaseView"); + return (BaseView) view; + } + + public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namespace) { + return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build(); + } + + public static LoadViewResponse createView( + ViewCatalog catalog, Namespace namespace, CreateViewRequest request) { + request.validate(); + + ViewBuilder viewBuilder = + catalog + .buildView(TableIdentifier.of(namespace, request.name())) + .withSchema(request.schema()) + .withProperties(request.properties()) + .withDefaultNamespace(request.viewVersion().defaultNamespace()) + .withDefaultCatalog(request.viewVersion().defaultCatalog()) + .withLocation(request.location()); + + Set unsupportedRepresentations = + request.viewVersion().representations().stream() + .filter(r -> !(r instanceof SQLViewRepresentation)) + .map(ViewRepresentation::type) + .collect(Collectors.toSet()); + + if (!unsupportedRepresentations.isEmpty()) { + throw new IllegalStateException( + String.format("Found unsupported view representations: %s", unsupportedRepresentations)); + } + + request.viewVersion().representations().stream() + .filter(SQLViewRepresentation.class::isInstance) + .map(SQLViewRepresentation.class::cast) + .forEach(r -> viewBuilder.withQuery(r.dialect(), r.sql())); + + View view = viewBuilder.create(); + + return viewResponse(view); + } + + private static LoadViewResponse viewResponse(View view) { + ViewMetadata metadata = asBaseView(view).operations().current(); + return ImmutableLoadViewResponse.builder() + .metadata(metadata) + .metadataLocation(metadata.metadataFileLocation()) + .build(); + } + + public static LoadViewResponse loadView(ViewCatalog catalog, TableIdentifier viewIdentifier) { + View view = catalog.loadView(viewIdentifier); + return viewResponse(view); + } + + public static LoadViewResponse updateView( + ViewCatalog catalog, TableIdentifier ident, UpdateTableRequest request) { + View view = catalog.loadView(ident); + ViewMetadata metadata = commit(asBaseView(view).operations(), request); + + return ImmutableLoadViewResponse.builder() + .metadata(metadata) + .metadataLocation(metadata.metadataFileLocation()) + .build(); + } + + public static void renameView(ViewCatalog catalog, RenameTableRequest request) { + catalog.renameView(request.source(), request.destination()); + } + + public static void dropView(ViewCatalog catalog, TableIdentifier viewIdentifier) { + boolean dropped = catalog.dropView(viewIdentifier); + if (!dropped) { + throw new NoSuchViewException("View does not exist: %s", viewIdentifier); + } + } + + static ViewMetadata commit(ViewOperations ops, UpdateTableRequest request) { + AtomicBoolean isRetry = new AtomicBoolean(false); + try { + Tasks.foreach(ops) + .retry(COMMIT_NUM_RETRIES_DEFAULT) + .exponentialBackoff( + COMMIT_MIN_RETRY_WAIT_MS_DEFAULT, + COMMIT_MAX_RETRY_WAIT_MS_DEFAULT, + COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT, + 2.0 /* exponential */) + .onlyRetryOn(CommitFailedException.class) + .run( + taskOps -> { + ViewMetadata base = isRetry.get() ? taskOps.refresh() : taskOps.current(); + isRetry.set(true); + + // validate requirements + try { + request.requirements().forEach(requirement -> requirement.validate(base)); + } catch (CommitFailedException e) { + // wrap and rethrow outside of tasks to avoid unnecessary retry + throw new ValidationFailureException(e); + } + + // apply changes + ViewMetadata.Builder metadataBuilder = ViewMetadata.buildFrom(base); + request.updates().forEach(update -> update.applyTo(metadataBuilder)); + + ViewMetadata updated = metadataBuilder.build(); + + if (updated.changes().isEmpty()) { + // do not commit if the metadata has not changed + return; + } + + // commit + taskOps.commit(base, updated); + }); + + } catch (ValidationFailureException e) { + throw e.wrapped(); + } + + return ops.current(); + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java index 5a7f9b59f2d4..846820a99d9f 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java @@ -26,6 +26,7 @@ import org.apache.iceberg.exceptions.ForbiddenException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.exceptions.NotAuthorizedException; import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.exceptions.ServiceFailureException; @@ -55,6 +56,14 @@ public static Consumer tableErrorHandler() { return TableErrorHandler.INSTANCE; } + public static Consumer viewErrorHandler() { + return ViewErrorHandler.INSTANCE; + } + + public static Consumer viewCommitHandler() { + return ViewCommitErrorHandler.INSTANCE; + } + public static Consumer tableCommitHandler() { return CommitErrorHandler.INSTANCE; } @@ -110,6 +119,49 @@ public void accept(ErrorResponse error) { } } + /** View commit error handler. */ + private static class ViewCommitErrorHandler extends DefaultErrorHandler { + private static final ErrorHandler INSTANCE = new ViewCommitErrorHandler(); + + @Override + public void accept(ErrorResponse error) { + switch (error.code()) { + case 404: + throw new NoSuchViewException("%s", error.message()); + case 409: + throw new CommitFailedException("Commit failed: %s", error.message()); + case 500: + case 502: + case 504: + throw new CommitStateUnknownException( + new ServiceFailureException("Service failed: %s: %s", error.code(), error.message())); + } + + super.accept(error); + } + } + + /** View level error handler. */ + private static class ViewErrorHandler extends DefaultErrorHandler { + private static final ErrorHandler INSTANCE = new ViewErrorHandler(); + + @Override + public void accept(ErrorResponse error) { + switch (error.code()) { + case 404: + if (NoSuchNamespaceException.class.getSimpleName().equals(error.type())) { + throw new NoSuchNamespaceException("%s", error.message()); + } else { + throw new NoSuchViewException("%s", error.message()); + } + case 409: + throw new AlreadyExistsException("%s", error.message()); + } + + super.accept(error); + } + } + /** Request error handler specifically for CRUD ops on namespaces. */ private static class NamespaceErrorHandler extends DefaultErrorHandler { private static final ErrorHandler INSTANCE = new NamespaceErrorHandler(); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java index 63b660c46aa3..61a7eca272df 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java @@ -35,17 +35,22 @@ import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableCommit; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.catalog.ViewCatalog; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewBuilder; -public class RESTCatalog implements Catalog, SupportsNamespaces, Configurable, Closeable { +public class RESTCatalog + implements Catalog, ViewCatalog, SupportsNamespaces, Configurable, Closeable { private final RESTSessionCatalog sessionCatalog; private final Catalog delegate; private final SupportsNamespaces nsDelegate; private final SessionCatalog.SessionContext context; + private final ViewCatalog viewSessionCatalog; public RESTCatalog() { this( @@ -64,6 +69,7 @@ public RESTCatalog( this.delegate = sessionCatalog.asCatalog(context); this.nsDelegate = (SupportsNamespaces) delegate; this.context = context; + this.viewSessionCatalog = sessionCatalog.asViewCatalog(context); } @Override @@ -261,4 +267,39 @@ public void commitTransaction(TableCommit... commits) { sessionCatalog.commitTransaction( context, ImmutableList.builder().add(commits).build()); } + + @Override + public List listViews(Namespace namespace) { + return viewSessionCatalog.listViews(namespace); + } + + @Override + public View loadView(TableIdentifier identifier) { + return viewSessionCatalog.loadView(identifier); + } + + @Override + public ViewBuilder buildView(TableIdentifier identifier) { + return viewSessionCatalog.buildView(identifier); + } + + @Override + public boolean dropView(TableIdentifier identifier) { + return viewSessionCatalog.dropView(identifier); + } + + @Override + public void renameView(TableIdentifier from, TableIdentifier to) { + viewSessionCatalog.renameView(from, to); + } + + @Override + public boolean viewExists(TableIdentifier identifier) { + return viewSessionCatalog.viewExists(identifier); + } + + @Override + public void invalidateView(TableIdentifier identifier) { + viewSessionCatalog.invalidateView(identifier); + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java index 06f2de32df0c..7d4f327b67df 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java @@ -44,6 +44,9 @@ import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.requests.CommitTransactionRequest; import org.apache.iceberg.rest.requests.CommitTransactionRequestParser; +import org.apache.iceberg.rest.requests.CreateViewRequest; +import org.apache.iceberg.rest.requests.CreateViewRequestParser; +import org.apache.iceberg.rest.requests.ImmutableCreateViewRequest; import org.apache.iceberg.rest.requests.ImmutableRegisterTableRequest; import org.apache.iceberg.rest.requests.ImmutableReportMetricsRequest; import org.apache.iceberg.rest.requests.RegisterTableRequest; @@ -56,6 +59,9 @@ import org.apache.iceberg.rest.requests.UpdateTableRequestParser; import org.apache.iceberg.rest.responses.ErrorResponse; import org.apache.iceberg.rest.responses.ErrorResponseParser; +import org.apache.iceberg.rest.responses.ImmutableLoadViewResponse; +import org.apache.iceberg.rest.responses.LoadViewResponse; +import org.apache.iceberg.rest.responses.LoadViewResponseParser; import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.apache.iceberg.util.JsonUtil; @@ -101,7 +107,16 @@ public static void registerAll(ObjectMapper mapper) { .addDeserializer(RegisterTableRequest.class, new RegisterTableRequestDeserializer<>()) .addSerializer(ImmutableRegisterTableRequest.class, new RegisterTableRequestSerializer<>()) .addDeserializer( - ImmutableRegisterTableRequest.class, new RegisterTableRequestDeserializer<>()); + ImmutableRegisterTableRequest.class, new RegisterTableRequestDeserializer<>()) + .addSerializer(CreateViewRequest.class, new CreateViewRequestSerializer<>()) + .addSerializer(ImmutableCreateViewRequest.class, new CreateViewRequestSerializer<>()) + .addDeserializer(CreateViewRequest.class, new CreateViewRequestDeserializer<>()) + .addDeserializer(ImmutableCreateViewRequest.class, new CreateViewRequestDeserializer<>()) + .addSerializer(LoadViewResponse.class, new LoadViewResponseSerializer<>()) + .addSerializer(ImmutableLoadViewResponse.class, new LoadViewResponseSerializer<>()) + .addDeserializer(LoadViewResponse.class, new LoadViewResponseDeserializer<>()) + .addDeserializer(ImmutableLoadViewResponse.class, new LoadViewResponseDeserializer<>()); + mapper.registerModule(module); } @@ -379,4 +394,38 @@ public T deserialize(JsonParser p, DeserializationContext context) throws IOExce return (T) RegisterTableRequestParser.fromJson(jsonNode); } } + + static class CreateViewRequestSerializer extends JsonSerializer { + @Override + public void serialize(T request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + CreateViewRequestParser.toJson(request, gen); + } + } + + static class CreateViewRequestDeserializer + extends JsonDeserializer { + @Override + public T deserialize(JsonParser p, DeserializationContext context) throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return (T) CreateViewRequestParser.fromJson(jsonNode); + } + } + + static class LoadViewResponseSerializer extends JsonSerializer { + @Override + public void serialize(T request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + LoadViewResponseParser.toJson(request, gen); + } + } + + static class LoadViewResponseDeserializer + extends JsonDeserializer { + @Override + public T deserialize(JsonParser p, DeserializationContext context) throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return (T) LoadViewResponseParser.fromJson(jsonNode); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index 72547eec3486..5a55afbfce22 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -38,6 +38,7 @@ import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.EnvironmentContext; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.MetadataTableUtils; import org.apache.iceberg.MetadataUpdate; @@ -49,13 +50,14 @@ import org.apache.iceberg.TableOperations; import org.apache.iceberg.Transaction; import org.apache.iceberg.Transactions; -import org.apache.iceberg.catalog.BaseSessionCatalog; +import org.apache.iceberg.catalog.BaseViewSessionCatalog; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableCommit; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.FileIO; @@ -65,12 +67,15 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.auth.OAuth2Properties; import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession; import org.apache.iceberg.rest.requests.CommitTransactionRequest; import org.apache.iceberg.rest.requests.CreateNamespaceRequest; import org.apache.iceberg.rest.requests.CreateTableRequest; +import org.apache.iceberg.rest.requests.CreateViewRequest; +import org.apache.iceberg.rest.requests.ImmutableCreateViewRequest; import org.apache.iceberg.rest.requests.ImmutableRegisterTableRequest; import org.apache.iceberg.rest.requests.RegisterTableRequest; import org.apache.iceberg.rest.requests.RenameTableRequest; @@ -82,15 +87,25 @@ import org.apache.iceberg.rest.responses.ListNamespacesResponse; import org.apache.iceberg.rest.responses.ListTablesResponse; import org.apache.iceberg.rest.responses.LoadTableResponse; +import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse; import org.apache.iceberg.util.EnvironmentUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.ThreadPools; +import org.apache.iceberg.view.BaseView; +import org.apache.iceberg.view.ImmutableSQLViewRepresentation; +import org.apache.iceberg.view.ImmutableViewVersion; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewBuilder; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewRepresentation; +import org.apache.iceberg.view.ViewUtil; +import org.apache.iceberg.view.ViewVersion; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class RESTSessionCatalog extends BaseSessionCatalog +public class RESTSessionCatalog extends BaseViewSessionCatalog implements Configurable, Closeable { private static final Logger LOG = LoggerFactory.getLogger(RESTSessionCatalog.class); private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO"; @@ -919,6 +934,12 @@ private void checkIdentifierIsValid(TableIdentifier tableIdentifier) { } } + private void checkViewIdentifierIsValid(TableIdentifier identifier) { + if (identifier.namespace().isEmpty()) { + throw new NoSuchViewException("Invalid view identifier: %s", identifier); + } + } + private void checkNamespaceIsValid(Namespace namespace) { if (namespace.isEmpty()) { throw new NoSuchNamespaceException("Invalid namespace: %s", namespace); @@ -971,4 +992,239 @@ public void commitTransaction(SessionContext context, List commits) headers(context), ErrorHandlers.tableCommitHandler()); } + + @Override + public List listViews(SessionContext context, Namespace namespace) { + checkNamespaceIsValid(namespace); + + ListTablesResponse response = + client.get( + paths.views(namespace), + ListTablesResponse.class, + headers(context), + ErrorHandlers.namespaceErrorHandler()); + return response.identifiers(); + } + + @Override + public View loadView(SessionContext context, TableIdentifier identifier) { + checkViewIdentifierIsValid(identifier); + + LoadViewResponse response = + client.get( + paths.view(identifier), + LoadViewResponse.class, + headers(context), + ErrorHandlers.viewErrorHandler()); + + AuthSession session = tableSession(response.config(), session(context)); + ViewMetadata metadata = response.metadata(); + + RESTViewOperations ops = + new RESTViewOperations(client, paths.view(identifier), session::headers, metadata); + + return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); + } + + @Override + public RESTViewBuilder buildView(SessionContext context, TableIdentifier identifier) { + return new RESTViewBuilder(context, identifier); + } + + @Override + public boolean dropView(SessionContext context, TableIdentifier identifier) { + checkViewIdentifierIsValid(identifier); + + try { + client.delete( + paths.view(identifier), null, headers(context), ErrorHandlers.viewErrorHandler()); + return true; + } catch (NoSuchViewException e) { + return false; + } + } + + @Override + public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) { + checkViewIdentifierIsValid(from); + checkViewIdentifierIsValid(to); + + RenameTableRequest request = + RenameTableRequest.builder().withSource(from).withDestination(to).build(); + + client.post( + paths.renameView(), request, null, headers(context), ErrorHandlers.viewErrorHandler()); + } + + private class RESTViewBuilder implements ViewBuilder { + private final SessionContext context; + private final TableIdentifier identifier; + private final Map properties = Maps.newHashMap(); + private final List representations = Lists.newArrayList(); + private Namespace defaultNamespace = null; + private String defaultCatalog = null; + private Schema schema = null; + private String location = null; + + private RESTViewBuilder(SessionContext context, TableIdentifier identifier) { + checkViewIdentifierIsValid(identifier); + this.identifier = identifier; + this.context = context; + } + + @Override + public ViewBuilder withSchema(Schema newSchema) { + this.schema = newSchema; + return this; + } + + @Override + public ViewBuilder withQuery(String dialect, String sql) { + representations.add( + ImmutableSQLViewRepresentation.builder().dialect(dialect).sql(sql).build()); + return this; + } + + @Override + public ViewBuilder withDefaultCatalog(String catalog) { + this.defaultCatalog = catalog; + return this; + } + + @Override + public ViewBuilder withDefaultNamespace(Namespace namespace) { + this.defaultNamespace = namespace; + return this; + } + + @Override + public ViewBuilder withProperties(Map newProperties) { + this.properties.putAll(newProperties); + return this; + } + + @Override + public ViewBuilder withProperty(String key, String value) { + this.properties.put(key, value); + return this; + } + + @Override + public ViewBuilder withLocation(String newLocation) { + this.location = newLocation; + return this; + } + + @Override + public View create() { + Preconditions.checkState( + !representations.isEmpty(), "Cannot create view without specifying a query"); + Preconditions.checkState(null != schema, "Cannot create view without specifying schema"); + Preconditions.checkState( + null != defaultNamespace, "Cannot create view without specifying a default namespace"); + + ViewVersion viewVersion = + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(schema.schemaId()) + .addAllRepresentations(representations) + .defaultNamespace(defaultNamespace) + .defaultCatalog(defaultCatalog) + .timestampMillis(System.currentTimeMillis()) + .putAllSummary(EnvironmentContext.get()) + .build(); + + CreateViewRequest request = + ImmutableCreateViewRequest.builder() + .name(identifier.name()) + .location(location) + .schema(schema) + .viewVersion(viewVersion) + .properties(properties) + .build(); + + LoadViewResponse response = + client.post( + paths.views(identifier.namespace()), + request, + LoadViewResponse.class, + headers(context), + ErrorHandlers.viewErrorHandler()); + + AuthSession session = tableSession(response.config(), session(context)); + RESTViewOperations ops = + new RESTViewOperations( + client, paths.view(identifier), session::headers, response.metadata()); + + return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); + } + + @Override + public View createOrReplace() { + try { + return replace(loadView()); + } catch (NoSuchViewException e) { + return create(); + } + } + + @Override + public View replace() { + return replace(loadView()); + } + + private LoadViewResponse loadView() { + return client.get( + paths.view(identifier), + LoadViewResponse.class, + headers(context), + ErrorHandlers.viewErrorHandler()); + } + + private View replace(LoadViewResponse response) { + Preconditions.checkState( + !representations.isEmpty(), "Cannot replace view without specifying a query"); + Preconditions.checkState(null != schema, "Cannot replace view without specifying schema"); + Preconditions.checkState( + null != defaultNamespace, "Cannot replace view without specifying a default namespace"); + + ViewMetadata metadata = response.metadata(); + + int maxVersionId = + metadata.versions().stream() + .map(ViewVersion::versionId) + .max(Integer::compareTo) + .orElseGet(metadata::currentVersionId); + + ViewVersion viewVersion = + ImmutableViewVersion.builder() + .versionId(maxVersionId + 1) + .schemaId(schema.schemaId()) + .addAllRepresentations(representations) + .defaultNamespace(defaultNamespace) + .defaultCatalog(defaultCatalog) + .timestampMillis(System.currentTimeMillis()) + .putAllSummary(EnvironmentContext.get()) + .build(); + + ViewMetadata.Builder builder = + ViewMetadata.buildFrom(metadata) + .setProperties(properties) + .setCurrentVersion(viewVersion, schema); + + if (null != location) { + builder.setLocation(location); + } + + ViewMetadata replacement = builder.build(); + + AuthSession session = tableSession(response.config(), session(context)); + RESTViewOperations ops = + new RESTViewOperations(client, paths.view(identifier), session::headers, metadata); + + ops.commit(metadata, replacement); + + return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java new file mode 100644 index 000000000000..48dc075b1305 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import java.util.Map; +import java.util.Objects; +import java.util.function.Supplier; +import org.apache.iceberg.UpdateRequirement; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.rest.requests.UpdateTableRequest; +import org.apache.iceberg.rest.responses.LoadViewResponse; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewOperations; + +class RESTViewOperations implements ViewOperations { + private final RESTClient client; + private final String path; + private final Supplier> headers; + private ViewMetadata current; + + RESTViewOperations( + RESTClient client, String path, Supplier> headers, ViewMetadata current) { + Preconditions.checkArgument(null != current, "Invalid view metadata: null"); + this.client = client; + this.path = path; + this.headers = headers; + this.current = current; + } + + @Override + public ViewMetadata current() { + return current; + } + + @Override + public ViewMetadata refresh() { + return updateCurrentMetadata( + client.get(path, LoadViewResponse.class, headers, ErrorHandlers.viewErrorHandler())); + } + + @Override + public void commit(ViewMetadata base, ViewMetadata metadata) { + // this is only used for replacing view metadata + Preconditions.checkState(base != null, "Invalid base metadata: null"); + + UpdateTableRequest request = + UpdateTableRequest.create( + null, + ImmutableList.of(new UpdateRequirement.AssertTableUUID(base.uuid())), + metadata.changes()); + + LoadViewResponse response = + client.post( + path, request, LoadViewResponse.class, headers, ErrorHandlers.viewCommitHandler()); + + updateCurrentMetadata(response); + } + + private ViewMetadata updateCurrentMetadata(LoadViewResponse response) { + if (!Objects.equals(current.metadataFileLocation(), response.metadataLocation())) { + this.current = response.metadata(); + } + + return current; + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java index b5b974f14c5e..c68a4f450843 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java +++ b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java @@ -93,4 +93,22 @@ public String metrics(TableIdentifier identifier) { public String commitTransaction() { return SLASH.join("v1", prefix, "transactions", "commit"); } + + public String views(Namespace ns) { + return SLASH.join("v1", prefix, "namespaces", RESTUtil.encodeNamespace(ns), "views"); + } + + public String view(TableIdentifier ident) { + return SLASH.join( + "v1", + prefix, + "namespaces", + RESTUtil.encodeNamespace(ident.namespace()), + "views", + RESTUtil.encodeString(ident.name())); + } + + public String renameView() { + return SLASH.join("v1", prefix, "views", "rename"); + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/CreateViewRequest.java b/core/src/main/java/org/apache/iceberg/rest/requests/CreateViewRequest.java new file mode 100644 index 000000000000..1c355273e08d --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/requests/CreateViewRequest.java @@ -0,0 +1,45 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.rest.RESTRequest; +import org.apache.iceberg.view.ViewVersion; +import org.immutables.value.Value; + +@Value.Immutable +public interface CreateViewRequest extends RESTRequest { + String name(); + + @Nullable + String location(); + + Schema schema(); + + ViewVersion viewVersion(); + + Map properties(); + + @Override + default void validate() { + // nothing to validate as it's not possible to create an invalid instance + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/CreateViewRequestParser.java b/core/src/main/java/org/apache/iceberg/rest/requests/CreateViewRequestParser.java new file mode 100644 index 000000000000..7a66bc4a1e8c --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/requests/CreateViewRequestParser.java @@ -0,0 +1,99 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.view.ViewVersion; +import org.apache.iceberg.view.ViewVersionParser; + +public class CreateViewRequestParser { + + private static final String NAME = "name"; + private static final String LOCATION = "location"; + private static final String SCHEMA = "schema"; + private static final String VIEW_VERSION = "view-version"; + private static final String PROPERTIES = "properties"; + + private CreateViewRequestParser() {} + + public static String toJson(CreateViewRequest request) { + return toJson(request, false); + } + + public static String toJson(CreateViewRequest request, boolean pretty) { + return JsonUtil.generate(gen -> toJson(request, gen), pretty); + } + + public static void toJson(CreateViewRequest request, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != request, "Invalid create view request: null"); + + gen.writeStartObject(); + + gen.writeStringField(NAME, request.name()); + + if (null != request.location()) { + gen.writeStringField(LOCATION, request.location()); + } + + gen.writeFieldName(VIEW_VERSION); + ViewVersionParser.toJson(request.viewVersion(), gen); + + gen.writeFieldName(SCHEMA); + SchemaParser.toJson(request.schema(), gen); + + if (!request.properties().isEmpty()) { + JsonUtil.writeStringMap(PROPERTIES, request.properties(), gen); + } + + gen.writeEndObject(); + } + + public static CreateViewRequest fromJson(String json) { + return JsonUtil.parse(json, CreateViewRequestParser::fromJson); + } + + public static CreateViewRequest fromJson(JsonNode json) { + Preconditions.checkArgument(null != json, "Cannot parse create view request from null object"); + + String name = JsonUtil.getString(NAME, json); + String location = JsonUtil.getStringOrNull(LOCATION, json); + + ViewVersion viewVersion = ViewVersionParser.fromJson(JsonUtil.get(VIEW_VERSION, json)); + Schema schema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, json)); + + ImmutableCreateViewRequest.Builder builder = + ImmutableCreateViewRequest.builder() + .name(name) + .location(location) + .viewVersion(viewVersion) + .schema(schema); + + if (json.has(PROPERTIES)) { + builder.properties(JsonUtil.getStringMap(PROPERTIES, json)); + } + + return builder.build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/requests/RenameTableRequest.java b/core/src/main/java/org/apache/iceberg/rest/requests/RenameTableRequest.java index bb44410f2bc6..fad0ea3cd236 100644 --- a/core/src/main/java/org/apache/iceberg/rest/requests/RenameTableRequest.java +++ b/core/src/main/java/org/apache/iceberg/rest/requests/RenameTableRequest.java @@ -23,7 +23,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.rest.RESTRequest; -/** A REST request to rename a table. */ +/** A REST request to rename a table or a view. */ public class RenameTableRequest implements RESTRequest { private TableIdentifier source; diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java new file mode 100644 index 000000000000..d07ba872fdaa --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponse.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.responses; + +import java.util.Map; +import org.apache.iceberg.rest.RESTResponse; +import org.apache.iceberg.view.ViewMetadata; +import org.immutables.value.Value; + +@Value.Immutable +public interface LoadViewResponse extends RESTResponse { + String metadataLocation(); + + ViewMetadata metadata(); + + Map config(); + + @Override + default void validate() { + // nothing to validate as it's not possible to create an invalid instance + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java new file mode 100644 index 000000000000..a8aaf17e5d76 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadViewResponseParser.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.responses; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewMetadataParser; + +public class LoadViewResponseParser { + + private static final String METADATA_LOCATION = "metadata-location"; + private static final String METADATA = "metadata"; + private static final String CONFIG = "config"; + + private LoadViewResponseParser() {} + + public static String toJson(LoadViewResponse response) { + return toJson(response, false); + } + + public static String toJson(LoadViewResponse response, boolean pretty) { + return JsonUtil.generate(gen -> toJson(response, gen), pretty); + } + + public static void toJson(LoadViewResponse response, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != response, "Invalid load view response: null"); + + gen.writeStartObject(); + + gen.writeStringField(METADATA_LOCATION, response.metadataLocation()); + + gen.writeFieldName(METADATA); + ViewMetadataParser.toJson(response.metadata(), gen); + + if (!response.config().isEmpty()) { + JsonUtil.writeStringMap(CONFIG, response.config(), gen); + } + + gen.writeEndObject(); + } + + public static LoadViewResponse fromJson(String json) { + return JsonUtil.parse(json, LoadViewResponseParser::fromJson); + } + + public static LoadViewResponse fromJson(JsonNode json) { + Preconditions.checkArgument(null != json, "Cannot parse load view response from null object"); + + String metadataLocation = JsonUtil.getString(METADATA_LOCATION, json); + ViewMetadata metadata = ViewMetadataParser.fromJson(JsonUtil.get(METADATA, json)); + + if (null == metadata.metadataFileLocation()) { + metadata = ViewMetadata.buildFrom(metadata).setMetadataLocation(metadataLocation).build(); + } + + ImmutableLoadViewResponse.Builder builder = + ImmutableLoadViewResponse.builder().metadataLocation(metadataLocation).metadata(metadata); + + if (json.has(CONFIG)) { + builder.config(JsonUtil.getStringMap(CONFIG, json)); + } + + return builder.build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java index 51921d476c4f..fa75c352f198 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java @@ -155,6 +155,7 @@ class Builder { // internal change tracking private Integer lastAddedVersionId = null; + private Integer lastAddedSchemaId = null; // indexes private final Map versionsById; @@ -257,8 +258,13 @@ public Builder addVersion(ViewVersion version) { return this; } - private int addVersionInternal(ViewVersion version) { - int newVersionId = reuseOrCreateNewViewVersionId(version); + private int addVersionInternal(ViewVersion newVersion) { + int newVersionId = reuseOrCreateNewViewVersionId(newVersion); + ViewVersion version = newVersion; + if (newVersionId != version.versionId()) { + version = ImmutableViewVersion.builder().from(version).versionId(newVersionId).build(); + } + if (versionsById.containsKey(newVersionId)) { boolean addedInBuilder = changes(MetadataUpdate.AddViewVersion.class) @@ -267,6 +273,13 @@ private int addVersionInternal(ViewVersion version) { return newVersionId; } + if (newVersion.schemaId() == LAST_ADDED) { + ValidationException.check( + lastAddedSchemaId != null, "Cannot set last added schema: no schema has been added"); + version = + ImmutableViewVersion.builder().from(newVersion).schemaId(lastAddedSchemaId).build(); + } + Preconditions.checkArgument( schemasById.containsKey(version.schemaId()), "Cannot add version with unknown schema: %s", @@ -283,20 +296,21 @@ private int addVersionInternal(ViewVersion version) { } } - ViewVersion newVersion; - if (newVersionId != version.versionId()) { - newVersion = ImmutableViewVersion.builder().from(version).versionId(newVersionId).build(); + versions.add(version); + versionsById.put(version.versionId(), version); + + if (null != lastAddedSchemaId && version.schemaId() == lastAddedSchemaId) { + changes.add( + new MetadataUpdate.AddViewVersion( + ImmutableViewVersion.builder().from(version).schemaId(LAST_ADDED).build())); } else { - newVersion = version; + changes.add(new MetadataUpdate.AddViewVersion(version)); } - versions.add(newVersion); - versionsById.put(newVersion.versionId(), newVersion); - changes.add(new MetadataUpdate.AddViewVersion(newVersion)); history.add( ImmutableViewHistoryEntry.builder() - .timestampMillis(newVersion.timestampMillis()) - .versionId(newVersion.versionId()) + .timestampMillis(version.timestampMillis()) + .versionId(version.versionId()) .build()); this.lastAddedVersionId = newVersionId; @@ -358,6 +372,8 @@ private int addSchemaInternal(Schema schema) { schemasById.put(newSchema.schemaId(), newSchema); changes.add(new MetadataUpdate.AddSchema(newSchema, highestFieldId)); + this.lastAddedSchemaId = newSchemaId; + return newSchemaId; } diff --git a/core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java b/core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java index f50a6c4a8517..5e7f2d9d1d11 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewMetadataParser.java @@ -62,7 +62,7 @@ public static String toJson(ViewMetadata metadata, boolean pretty) { return JsonUtil.generate(gen -> toJson(metadata, gen), pretty); } - static void toJson(ViewMetadata metadata, JsonGenerator gen) throws IOException { + public static void toJson(ViewMetadata metadata, JsonGenerator gen) throws IOException { Preconditions.checkArgument(null != metadata, "Invalid view metadata: null"); gen.writeStartObject(); diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java index 8bdbdc431c98..2645e40d94ef 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java @@ -65,7 +65,7 @@ public static void toJson(ViewVersion version, JsonGenerator generator) throws I generator.writeEndObject(); } - static String toJson(ViewVersion version) { + public static String toJson(ViewVersion version) { return JsonUtil.generate(gen -> toJson(version, gen), false); } diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java index c1ac43e8292a..8b3d087940a5 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionReplace.java @@ -55,7 +55,7 @@ public ViewVersion apply() { return internalApply().currentVersion(); } - private ViewMetadata internalApply() { + ViewMetadata internalApply() { Preconditions.checkState( !representations.isEmpty(), "Cannot replace view without specifying a query"); Preconditions.checkState(null != schema, "Cannot replace view without specifying schema"); diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java index 0772601b77df..1974838ede32 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -31,6 +31,7 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.catalog.ViewCatalog; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; @@ -39,6 +40,7 @@ import org.apache.iceberg.exceptions.NoSuchIcebergTableException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.exceptions.NotAuthorizedException; import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.exceptions.UnprocessableEntityException; @@ -49,6 +51,7 @@ import org.apache.iceberg.rest.requests.CommitTransactionRequest; import org.apache.iceberg.rest.requests.CreateNamespaceRequest; import org.apache.iceberg.rest.requests.CreateTableRequest; +import org.apache.iceberg.rest.requests.CreateViewRequest; import org.apache.iceberg.rest.requests.RegisterTableRequest; import org.apache.iceberg.rest.requests.RenameTableRequest; import org.apache.iceberg.rest.requests.ReportMetricsRequest; @@ -61,6 +64,7 @@ import org.apache.iceberg.rest.responses.ListNamespacesResponse; import org.apache.iceberg.rest.responses.ListTablesResponse; import org.apache.iceberg.rest.responses.LoadTableResponse; +import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse; import org.apache.iceberg.util.Pair; @@ -79,6 +83,7 @@ public class RESTCatalogAdapter implements RESTClient { .put(ForbiddenException.class, 403) .put(NoSuchNamespaceException.class, 404) .put(NoSuchTableException.class, 404) + .put(NoSuchViewException.class, 404) .put(NoSuchIcebergTableException.class, 404) .put(UnsupportedOperationException.class, 406) .put(AlreadyExistsException.class, 409) @@ -89,11 +94,13 @@ public class RESTCatalogAdapter implements RESTClient { private final Catalog catalog; private final SupportsNamespaces asNamespaceCatalog; + private final ViewCatalog asViewCatalog; public RESTCatalogAdapter(Catalog catalog) { this.catalog = catalog; this.asNamespaceCatalog = catalog instanceof SupportsNamespaces ? (SupportsNamespaces) catalog : null; + this.asViewCatalog = catalog instanceof ViewCatalog ? (ViewCatalog) catalog : null; } enum HTTPMethod { @@ -126,7 +133,7 @@ enum Route { CreateTableRequest.class, LoadTableResponse.class), LOAD_TABLE( - HTTPMethod.GET, "v1/namespaces/{namespace}/tables/{table}", null, LoadTableResponse.class), + HTTPMethod.GET, "v1/namespaces/{namespace}/tables/{name}", null, LoadTableResponse.class), REGISTER_TABLE( HTTPMethod.POST, "v1/namespaces/{namespace}/register", @@ -134,18 +141,33 @@ enum Route { LoadTableResponse.class), UPDATE_TABLE( HTTPMethod.POST, - "v1/namespaces/{namespace}/tables/{table}", + "v1/namespaces/{namespace}/tables/{name}", UpdateTableRequest.class, LoadTableResponse.class), - DROP_TABLE(HTTPMethod.DELETE, "v1/namespaces/{namespace}/tables/{table}"), + DROP_TABLE(HTTPMethod.DELETE, "v1/namespaces/{namespace}/tables/{name}"), RENAME_TABLE(HTTPMethod.POST, "v1/tables/rename", RenameTableRequest.class, null), REPORT_METRICS( HTTPMethod.POST, - "v1/namespaces/{namespace}/tables/{table}/metrics", + "v1/namespaces/{namespace}/tables/{name}/metrics", ReportMetricsRequest.class, null), COMMIT_TRANSACTION( - HTTPMethod.POST, "v1/transactions/commit", CommitTransactionRequest.class, null); + HTTPMethod.POST, "v1/transactions/commit", CommitTransactionRequest.class, null), + LIST_VIEWS(HTTPMethod.GET, "v1/namespaces/{namespace}/views", null, ListTablesResponse.class), + LOAD_VIEW( + HTTPMethod.GET, "v1/namespaces/{namespace}/views/{name}", null, LoadViewResponse.class), + CREATE_VIEW( + HTTPMethod.POST, + "v1/namespaces/{namespace}/views", + CreateViewRequest.class, + LoadViewResponse.class), + UPDATE_VIEW( + HTTPMethod.POST, + "v1/namespaces/{namespace}/views/{name}", + UpdateTableRequest.class, + LoadViewResponse.class), + RENAME_VIEW(HTTPMethod.POST, "v1/views/rename", RenameTableRequest.class, null), + DROP_VIEW(HTTPMethod.DELETE, "v1/namespaces/{namespace}/views/{name}"); private final HTTPMethod method; private final int requiredLength; @@ -223,7 +245,7 @@ public Class responseClass() { } } - @SuppressWarnings("MethodLength") + @SuppressWarnings({"MethodLength", "checkstyle:CyclomaticComplexity"}) public T handleRequest( Route route, Map vars, Object body, Class responseType) { switch (route) { @@ -387,6 +409,65 @@ public T handleRequest( return null; } + case LIST_VIEWS: + { + if (null != asViewCatalog) { + Namespace namespace = namespaceFromPathVars(vars); + return castResponse(responseType, CatalogHandlers.listViews(asViewCatalog, namespace)); + } + break; + } + + case CREATE_VIEW: + { + if (null != asViewCatalog) { + Namespace namespace = namespaceFromPathVars(vars); + CreateViewRequest request = castRequest(CreateViewRequest.class, body); + return castResponse( + responseType, CatalogHandlers.createView(asViewCatalog, namespace, request)); + } + break; + } + + case LOAD_VIEW: + { + if (null != asViewCatalog) { + TableIdentifier ident = identFromPathVars(vars); + return castResponse(responseType, CatalogHandlers.loadView(asViewCatalog, ident)); + } + break; + } + + case UPDATE_VIEW: + { + if (null != asViewCatalog) { + TableIdentifier ident = identFromPathVars(vars); + UpdateTableRequest request = castRequest(UpdateTableRequest.class, body); + return castResponse( + responseType, CatalogHandlers.updateView(asViewCatalog, ident, request)); + } + break; + } + + case RENAME_VIEW: + { + if (null != asViewCatalog) { + RenameTableRequest request = castRequest(RenameTableRequest.class, body); + CatalogHandlers.renameView(asViewCatalog, request); + return null; + } + break; + } + + case DROP_VIEW: + { + if (null != asViewCatalog) { + CatalogHandlers.dropView(asViewCatalog, identFromPathVars(vars)); + return null; + } + break; + } + default: } @@ -566,6 +647,6 @@ private static Namespace namespaceFromPathVars(Map pathVars) { private static TableIdentifier identFromPathVars(Map pathVars) { return TableIdentifier.of( - namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("table"))); + namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("name"))); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java new file mode 100644 index 000000000000..0b29da70426b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -0,0 +1,166 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.File; +import java.nio.file.Path; +import java.util.Map; +import java.util.UUID; +import java.util.function.Consumer; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.SessionCatalog; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.RESTCatalogAdapter.HTTPMethod; +import org.apache.iceberg.rest.responses.ErrorResponse; +import org.apache.iceberg.view.ViewCatalogTests; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; + +public class TestRESTViewCatalog extends ViewCatalogTests { + private static final ObjectMapper MAPPER = RESTObjectMapper.mapper(); + + @TempDir private Path temp; + + private RESTCatalog restCatalog; + private InMemoryCatalog backendCatalog; + private Server httpServer; + + @BeforeEach + public void createCatalog() throws Exception { + File warehouse = temp.toFile(); + + this.backendCatalog = new InMemoryCatalog(); + this.backendCatalog.initialize( + "in-memory", + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse.getAbsolutePath())); + + RESTCatalogAdapter adaptor = + new RESTCatalogAdapter(backendCatalog) { + @Override + public T execute( + HTTPMethod method, + String path, + Map queryParams, + Object body, + Class responseType, + Map headers, + Consumer errorHandler) { + Object request = roundTripSerialize(body, "request"); + T response = + super.execute( + method, path, queryParams, request, responseType, headers, errorHandler); + T responseAfterSerialization = roundTripSerialize(response, "response"); + return responseAfterSerialization; + } + }; + + RESTCatalogServlet servlet = new RESTCatalogServlet(adaptor); + ServletContextHandler servletContext = + new ServletContextHandler(ServletContextHandler.NO_SESSIONS); + servletContext.setContextPath("/"); + ServletHolder servletHolder = new ServletHolder(servlet); + servletHolder.setInitParameter("javax.ws.rs.Application", "ServiceListPublic"); + servletContext.addServlet(servletHolder, "/*"); + servletContext.setVirtualHosts(null); + servletContext.setGzipHandler(new GzipHandler()); + + this.httpServer = new Server(0); + httpServer.setHandler(servletContext); + httpServer.start(); + + SessionCatalog.SessionContext context = + new SessionCatalog.SessionContext( + UUID.randomUUID().toString(), + "user", + ImmutableMap.of("credential", "user:12345"), + ImmutableMap.of()); + + this.restCatalog = + new RESTCatalog( + context, + (config) -> HTTPClient.builder(config).uri(config.get(CatalogProperties.URI)).build()); + restCatalog.initialize( + "prod", + ImmutableMap.of( + CatalogProperties.URI, httpServer.getURI().toString(), "credential", "catalog:12345")); + } + + @SuppressWarnings("unchecked") + public static T roundTripSerialize(T payload, String description) { + if (payload != null) { + try { + if (payload instanceof RESTMessage) { + return (T) MAPPER.readValue(MAPPER.writeValueAsString(payload), payload.getClass()); + } else { + // use Map so that Jackson doesn't try to instantiate ImmutableMap from payload.getClass() + return (T) MAPPER.readValue(MAPPER.writeValueAsString(payload), Map.class); + } + } catch (JsonProcessingException e) { + throw new RuntimeException( + String.format("Failed to serialize and deserialize %s: %s", description, payload), e); + } + } + return null; + } + + @AfterEach + public void closeCatalog() throws Exception { + if (restCatalog != null) { + restCatalog.close(); + } + + if (backendCatalog != null) { + backendCatalog.close(); + } + + if (httpServer != null) { + httpServer.stop(); + httpServer.join(); + } + } + + @Override + protected RESTCatalog catalog() { + return restCatalog; + } + + @Override + protected Catalog tableCatalog() { + return restCatalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return true; + } + + @Override + protected boolean supportsServerSideRetry() { + return true; + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java index e0e61a594e7d..4b91fbbad380 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java @@ -143,4 +143,51 @@ public void testRegister() { .isEqualTo("v1/ws/catalog/namespaces/ns/register"); Assertions.assertThat(withoutPrefix.register(ns)).isEqualTo("v1/namespaces/ns/register"); } + + @Test + public void views() { + Namespace ns = Namespace.of("ns"); + Assertions.assertThat(withPrefix.views(ns)).isEqualTo("v1/ws/catalog/namespaces/ns/views"); + Assertions.assertThat(withoutPrefix.views(ns)).isEqualTo("v1/namespaces/ns/views"); + } + + @Test + public void viewsWithSlash() { + Namespace ns = Namespace.of("n/s"); + Assertions.assertThat(withPrefix.views(ns)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs/views"); + Assertions.assertThat(withoutPrefix.views(ns)).isEqualTo("v1/namespaces/n%2Fs/views"); + } + + @Test + public void viewsWithMultipartNamespace() { + Namespace ns = Namespace.of("n", "s"); + Assertions.assertThat(withPrefix.views(ns)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs/views"); + Assertions.assertThat(withoutPrefix.views(ns)).isEqualTo("v1/namespaces/n%1Fs/views"); + } + + @Test + public void view() { + TableIdentifier ident = TableIdentifier.of("ns", "view-name"); + Assertions.assertThat(withPrefix.view(ident)) + .isEqualTo("v1/ws/catalog/namespaces/ns/views/view-name"); + Assertions.assertThat(withoutPrefix.view(ident)).isEqualTo("v1/namespaces/ns/views/view-name"); + } + + @Test + public void viewWithSlash() { + TableIdentifier ident = TableIdentifier.of("n/s", "vi/ew-name"); + Assertions.assertThat(withPrefix.view(ident)) + .isEqualTo("v1/ws/catalog/namespaces/n%2Fs/views/vi%2Few-name"); + Assertions.assertThat(withoutPrefix.view(ident)) + .isEqualTo("v1/namespaces/n%2Fs/views/vi%2Few-name"); + } + + @Test + public void viewWithMultipartNamespace() { + TableIdentifier ident = TableIdentifier.of("n", "s", "view-name"); + Assertions.assertThat(withPrefix.view(ident)) + .isEqualTo("v1/ws/catalog/namespaces/n%1Fs/views/view-name"); + Assertions.assertThat(withoutPrefix.view(ident)) + .isEqualTo("v1/namespaces/n%1Fs/views/view-name"); + } } diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateViewRequestParser.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateViewRequestParser.java new file mode 100644 index 000000000000..a228c94a08ac --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateViewRequestParser.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.requests; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.ImmutableViewVersion; +import org.apache.iceberg.view.ViewVersionParser; +import org.junit.jupiter.api.Test; + +public class TestCreateViewRequestParser { + + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> CreateViewRequestParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid create view request: null"); + + assertThatThrownBy(() -> CreateViewRequestParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse create view request from null object"); + + assertThatThrownBy(() -> CreateViewRequestParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: name"); + } + + @Test + public void missingFields() { + assertThatThrownBy(() -> CreateViewRequestParser.fromJson("{\"x\": \"val\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: name"); + + assertThatThrownBy(() -> CreateViewRequestParser.fromJson("{\"name\": \"view-name\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: view-version"); + + String viewVersion = + ViewVersionParser.toJson( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(1) + .timestampMillis(23L) + .defaultNamespace(Namespace.of("ns1")) + .build()); + + assertThatThrownBy( + () -> + CreateViewRequestParser.fromJson( + String.format( + "{\"name\": \"view-name\", \"location\": \"loc\", \"view-version\": %s}", + viewVersion))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: schema"); + } + + @Test + public void roundTripSerde() { + CreateViewRequest request = + ImmutableCreateViewRequest.builder() + .name("view-name") + .viewVersion( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(1) + .timestampMillis(23L) + .defaultNamespace(Namespace.of("ns1")) + .build()) + .location("location") + .schema(new Schema(Types.NestedField.required(1, "x", Types.LongType.get()))) + .properties(ImmutableMap.of("key1", "val1")) + .build(); + + String expectedJson = + "{\n" + + " \"name\" : \"view-name\",\n" + + " \"location\" : \"location\",\n" + + " \"view-version\" : {\n" + + " \"version-id\" : 1,\n" + + " \"timestamp-ms\" : 23,\n" + + " \"schema-id\" : 0,\n" + + " \"summary\" : { },\n" + + " \"default-namespace\" : [ \"ns1\" ],\n" + + " \"representations\" : [ ]\n" + + " },\n" + + " \"schema\" : {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"x\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"long\"\n" + + " } ]\n" + + " },\n" + + " \"properties\" : {\n" + + " \"key1\" : \"val1\"\n" + + " }\n" + + "}"; + + String json = CreateViewRequestParser.toJson(request, true); + assertThat(json).isEqualTo(expectedJson); + + // can't do an equality comparison because Schema doesn't implement equals/hashCode + assertThat(CreateViewRequestParser.toJson(CreateViewRequestParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java new file mode 100644 index 000000000000..d94d035596c9 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadViewResponseParser.java @@ -0,0 +1,260 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.rest.responses; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.databind.JsonNode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.ImmutableViewVersion; +import org.apache.iceberg.view.ViewMetadata; +import org.junit.jupiter.api.Test; + +public class TestLoadViewResponseParser { + + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> LoadViewResponseParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid load view response: null"); + + assertThatThrownBy(() -> LoadViewResponseParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse load view response from null object"); + + assertThatThrownBy(() -> LoadViewResponseParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: metadata-location"); + } + + @Test + public void missingFields() { + assertThatThrownBy(() -> LoadViewResponseParser.fromJson("{\"x\": \"val\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing string: metadata-location"); + + assertThatThrownBy( + () -> LoadViewResponseParser.fromJson("{\"metadata-location\": \"custom-location\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: metadata"); + } + + @Test + public void roundTripSerde() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + ViewMetadata viewMetadata = + ViewMetadata.builder() + .assignUUID(uuid) + .setLocation("location") + .addSchema(new Schema(Types.NestedField.required(1, "x", Types.LongType.get()))) + .addVersion( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(1) + .timestampMillis(23L) + .defaultNamespace(Namespace.of("ns1")) + .build()) + .addVersion( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(2) + .timestampMillis(24L) + .defaultNamespace(Namespace.of("ns2")) + .build()) + .addVersion( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(3) + .timestampMillis(25L) + .defaultNamespace(Namespace.of("ns3")) + .build()) + .setCurrentVersionId(3) + .build(); + + LoadViewResponse response = + ImmutableLoadViewResponse.builder() + .metadata(viewMetadata) + .metadataLocation("custom-location") + .build(); + String expectedJson = + "{\n" + + " \"metadata-location\" : \"custom-location\",\n" + + " \"metadata\" : {\n" + + " \"view-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" + + " \"format-version\" : 1,\n" + + " \"location\" : \"location\",\n" + + " \"schemas\" : [ {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"x\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"long\"\n" + + " } ]\n" + + " } ],\n" + + " \"current-version-id\" : 3,\n" + + " \"versions\" : [ {\n" + + " \"version-id\" : 1,\n" + + " \"timestamp-ms\" : 23,\n" + + " \"schema-id\" : 0,\n" + + " \"summary\" : { },\n" + + " \"default-namespace\" : [ \"ns1\" ],\n" + + " \"representations\" : [ ]\n" + + " }, {\n" + + " \"version-id\" : 2,\n" + + " \"timestamp-ms\" : 24,\n" + + " \"schema-id\" : 0,\n" + + " \"summary\" : { },\n" + + " \"default-namespace\" : [ \"ns2\" ],\n" + + " \"representations\" : [ ]\n" + + " }, {\n" + + " \"version-id\" : 3,\n" + + " \"timestamp-ms\" : 25,\n" + + " \"schema-id\" : 0,\n" + + " \"summary\" : { },\n" + + " \"default-namespace\" : [ \"ns3\" ],\n" + + " \"representations\" : [ ]\n" + + " } ],\n" + + " \"version-log\" : [ {\n" + + " \"timestamp-ms\" : 23,\n" + + " \"version-id\" : 1\n" + + " }, {\n" + + " \"timestamp-ms\" : 24,\n" + + " \"version-id\" : 2\n" + + " }, {\n" + + " \"timestamp-ms\" : 25,\n" + + " \"version-id\" : 3\n" + + " } ]\n" + + " }\n" + + "}"; + + String json = LoadViewResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + // can't do an equality comparison because Schema doesn't implement equals/hashCode + assertThat(LoadViewResponseParser.toJson(LoadViewResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void roundTripSerdeWithConfig() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + ViewMetadata viewMetadata = + ViewMetadata.builder() + .assignUUID(uuid) + .setLocation("location") + .addSchema(new Schema(Types.NestedField.required(1, "x", Types.LongType.get()))) + .addVersion( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(1) + .timestampMillis(23L) + .defaultNamespace(Namespace.of("ns1")) + .build()) + .addVersion( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(2) + .timestampMillis(24L) + .defaultNamespace(Namespace.of("ns2")) + .build()) + .addVersion( + ImmutableViewVersion.builder() + .schemaId(0) + .versionId(3) + .timestampMillis(25L) + .defaultNamespace(Namespace.of("ns3")) + .build()) + .setCurrentVersionId(3) + .build(); + + LoadViewResponse response = + ImmutableLoadViewResponse.builder() + .metadata(viewMetadata) + .metadataLocation("custom-location") + .config(ImmutableMap.of("key1", "val1", "key2", "val2")) + .build(); + String expectedJson = + "{\n" + + " \"metadata-location\" : \"custom-location\",\n" + + " \"metadata\" : {\n" + + " \"view-uuid\" : \"386b9f01-002b-4d8c-b77f-42c3fd3b7c9b\",\n" + + " \"format-version\" : 1,\n" + + " \"location\" : \"location\",\n" + + " \"schemas\" : [ {\n" + + " \"type\" : \"struct\",\n" + + " \"schema-id\" : 0,\n" + + " \"fields\" : [ {\n" + + " \"id\" : 1,\n" + + " \"name\" : \"x\",\n" + + " \"required\" : true,\n" + + " \"type\" : \"long\"\n" + + " } ]\n" + + " } ],\n" + + " \"current-version-id\" : 3,\n" + + " \"versions\" : [ {\n" + + " \"version-id\" : 1,\n" + + " \"timestamp-ms\" : 23,\n" + + " \"schema-id\" : 0,\n" + + " \"summary\" : { },\n" + + " \"default-namespace\" : [ \"ns1\" ],\n" + + " \"representations\" : [ ]\n" + + " }, {\n" + + " \"version-id\" : 2,\n" + + " \"timestamp-ms\" : 24,\n" + + " \"schema-id\" : 0,\n" + + " \"summary\" : { },\n" + + " \"default-namespace\" : [ \"ns2\" ],\n" + + " \"representations\" : [ ]\n" + + " }, {\n" + + " \"version-id\" : 3,\n" + + " \"timestamp-ms\" : 25,\n" + + " \"schema-id\" : 0,\n" + + " \"summary\" : { },\n" + + " \"default-namespace\" : [ \"ns3\" ],\n" + + " \"representations\" : [ ]\n" + + " } ],\n" + + " \"version-log\" : [ {\n" + + " \"timestamp-ms\" : 23,\n" + + " \"version-id\" : 1\n" + + " }, {\n" + + " \"timestamp-ms\" : 24,\n" + + " \"version-id\" : 2\n" + + " }, {\n" + + " \"timestamp-ms\" : 25,\n" + + " \"version-id\" : 3\n" + + " } ]\n" + + " },\n" + + " \"config\" : {\n" + + " \"key1\" : \"val1\",\n" + + " \"key2\" : \"val2\"\n" + + " }\n" + + "}"; + + String json = LoadViewResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + // can't do an equality comparison because Schema doesn't implement equals/hashCode + assertThat(LoadViewResponseParser.toJson(LoadViewResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } +} diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java index 0f1758f85e08..e60fe3b285b5 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java @@ -28,6 +28,7 @@ import org.apache.iceberg.MetadataUpdate; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; @@ -289,21 +290,21 @@ public void viewHistoryNormalization() { .isInstanceOf(MetadataUpdate.AddViewVersion.class) .asInstanceOf(InstanceOfAssertFactories.type(MetadataUpdate.AddViewVersion.class)) .extracting(MetadataUpdate.AddViewVersion::viewVersion) - .isEqualTo(viewVersionOne); + .isEqualTo(ImmutableViewVersion.builder().from(viewVersionOne).schemaId(-1).build()); assertThat(changes) .element(4) .isInstanceOf(MetadataUpdate.AddViewVersion.class) .asInstanceOf(InstanceOfAssertFactories.type(MetadataUpdate.AddViewVersion.class)) .extracting(MetadataUpdate.AddViewVersion::viewVersion) - .isEqualTo(viewVersionTwo); + .isEqualTo(ImmutableViewVersion.builder().from(viewVersionTwo).schemaId(-1).build()); assertThat(changes) .element(5) .isInstanceOf(MetadataUpdate.AddViewVersion.class) .asInstanceOf(InstanceOfAssertFactories.type(MetadataUpdate.AddViewVersion.class)) .extracting(MetadataUpdate.AddViewVersion::viewVersion) - .isEqualTo(viewVersionThree); + .isEqualTo(ImmutableViewVersion.builder().from(viewVersionThree).schemaId(-1).build()); assertThat(changes) .element(6) @@ -409,7 +410,7 @@ public void viewMetadataAndMetadataChanges() { .isInstanceOf(MetadataUpdate.AddViewVersion.class) .asInstanceOf(InstanceOfAssertFactories.type(MetadataUpdate.AddViewVersion.class)) .extracting(MetadataUpdate.AddViewVersion::viewVersion) - .isEqualTo(viewVersionThree); + .isEqualTo(ImmutableViewVersion.builder().from(viewVersionThree).schemaId(-1).build()); assertThat(changes) .element(8) @@ -784,4 +785,18 @@ public void viewMetadataWithMultipleSQLForSameDialect() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid view version: Cannot add multiple queries for dialect spark"); } + + @Test + public void lastAddedSchemaFailure() { + ViewVersion viewVersion = newViewVersion(1, -1, "select * from ns.tbl"); + assertThatThrownBy( + () -> + ViewMetadata.builder() + .setLocation("custom-location") + .addVersion(viewVersion) + .setCurrentVersionId(1) + .build()) + .isInstanceOf(ValidationException.class) + .hasMessage("Cannot set last added schema: no schema has been added"); + } } diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 682d7ade67e8..8cb77a776220 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -34,6 +34,7 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.catalog.ViewCatalog; import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; @@ -69,6 +70,10 @@ protected boolean overridesRequestedLocation() { return false; } + protected boolean supportsServerSideRetry() { + return false; + } + @Test public void basicCreateView() { TableIdentifier identifier = TableIdentifier.of("ns", "view"); @@ -247,8 +252,9 @@ public void createViewErrorCases() { .withQuery(trino.dialect(), trino.sql()) .withQuery(trino.dialect(), trino.sql()) .create()) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid view version: Cannot add multiple queries for dialect trino"); + .isInstanceOf(Exception.class) + .hasMessageContaining( + "Invalid view version: Cannot add multiple queries for dialect trino"); } @Test @@ -1534,4 +1540,138 @@ public void updateViewLocationConflict() { .isInstanceOf(NoSuchViewException.class) .hasMessageContaining("View does not exist: ns.view"); } + + @Test + public void concurrentReplaceViewVersion() { + TableIdentifier identifier = TableIdentifier.of("ns", "view"); + + if (requiresNamespaceCreate()) { + catalog().createNamespace(identifier.namespace()); + } + + assertThat(catalog().viewExists(identifier)).as("View should not exist").isFalse(); + + View view = + catalog() + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()) + .withQuery("trino", "select * from ns.tbl") + .create(); + + assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); + + ReplaceViewVersion replaceViewVersionOne = + view.replaceVersion() + .withQuery("trino", "select count(id) from ns.tbl") + .withSchema(SCHEMA) + .withDefaultNamespace(identifier.namespace()); + + ReplaceViewVersion replaceViewVersionTwo = + view.replaceVersion() + .withQuery("spark", "select count(some_id) from ns.tbl") + .withSchema(OTHER_SCHEMA) + .withDefaultNamespace(identifier.namespace()); + + // simulate a concurrent replace of the view version + ViewOperations viewOps = ((BaseView) view).operations(); + ViewMetadata current = viewOps.current(); + + ViewMetadata trinoUpdate = ((ViewVersionReplace) replaceViewVersionTwo).internalApply(); + ViewMetadata sparkUpdate = ((ViewVersionReplace) replaceViewVersionOne).internalApply(); + + viewOps.commit(current, trinoUpdate); + + if (supportsServerSideRetry()) { + // retry should succeed and the changes should be applied + viewOps.commit(current, sparkUpdate); + + View updatedView = catalog().loadView(identifier); + ViewVersion viewVersion = updatedView.currentVersion(); + assertThat(viewVersion.versionId()).isEqualTo(3); + assertThat(updatedView.versions()).hasSize(3); + assertThat(updatedView.version(1)) + .isEqualTo( + ImmutableViewVersion.builder() + .timestampMillis(updatedView.version(1).timestampMillis()) + .versionId(1) + .schemaId(0) + .summary(updatedView.version(1).summary()) + .defaultNamespace(identifier.namespace()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("trino") + .build()) + .build()); + + assertThat(updatedView.version(2)) + .isEqualTo( + ImmutableViewVersion.builder() + .timestampMillis(updatedView.version(2).timestampMillis()) + .versionId(2) + .schemaId(1) + .summary(updatedView.version(2).summary()) + .defaultNamespace(identifier.namespace()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select count(some_id) from ns.tbl") + .dialect("spark") + .build()) + .build()); + + assertThat(updatedView.version(3)) + .isEqualTo( + ImmutableViewVersion.builder() + .timestampMillis(updatedView.version(3).timestampMillis()) + .versionId(3) + .schemaId(0) + .summary(updatedView.version(3).summary()) + .defaultNamespace(identifier.namespace()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select count(id) from ns.tbl") + .dialect("trino") + .build()) + .build()); + } else { + assertThatThrownBy(() -> viewOps.commit(current, sparkUpdate)) + .isInstanceOf(CommitFailedException.class) + .hasMessageContaining("Cannot commit"); + + View updatedView = catalog().loadView(identifier); + ViewVersion viewVersion = updatedView.currentVersion(); + assertThat(viewVersion.versionId()).isEqualTo(2); + assertThat(updatedView.versions()).hasSize(2); + assertThat(updatedView.version(1)) + .isEqualTo( + ImmutableViewVersion.builder() + .timestampMillis(updatedView.version(1).timestampMillis()) + .versionId(1) + .schemaId(0) + .summary(updatedView.version(1).summary()) + .defaultNamespace(identifier.namespace()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select * from ns.tbl") + .dialect("trino") + .build()) + .build()); + + assertThat(updatedView.version(2)) + .isEqualTo( + ImmutableViewVersion.builder() + .timestampMillis(updatedView.version(2).timestampMillis()) + .versionId(2) + .schemaId(1) + .summary(updatedView.version(2).summary()) + .defaultNamespace(identifier.namespace()) + .addRepresentations( + ImmutableSQLViewRepresentation.builder() + .sql("select count(some_id) from ns.tbl") + .dialect("spark") + .build()) + .build()); + } + } } diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 4db1fccdc2b0..5da91a16f901 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -209,6 +209,35 @@ class MetadataLog(BaseModel): __root__: List[MetadataLogItem] +class SQLViewRepresentation(BaseModel): + type: str + sql: str + dialect: str + + +class ViewRepresentation(BaseModel): + __root__: SQLViewRepresentation + + +class ViewHistoryEntry(BaseModel): + version_id: int = Field(..., alias='version-id') + timestamp_ms: int = Field(..., alias='timestamp-ms') + + +class ViewVersion(BaseModel): + version_id: int = Field(..., alias='version-id') + timestamp_ms: int = Field(..., alias='timestamp-ms') + schema_id: int = Field( + ..., + alias='schema-id', + description='Schema ID to set as current, or -1 to set last added schema', + ) + summary: Dict[str, str] + representations: List[ViewRepresentation] + default_catalog: Optional[str] = Field(None, alias='default-catalog') + default_namespace: Namespace = Field(..., alias='default-namespace') + + class BaseUpdate(BaseModel): action: Literal[ 'assign-uuid', @@ -226,6 +255,8 @@ class BaseUpdate(BaseModel): 'set-location', 'set-properties', 'remove-properties', + 'add-view-version', + 'set-current-view-version', ] @@ -301,6 +332,18 @@ class RemovePropertiesUpdate(BaseUpdate): removals: List[str] +class AddViewVersionUpdate(BaseUpdate): + view_version: ViewVersion = Field(..., alias='view-version') + + +class SetCurrentViewVersionUpdate(BaseUpdate): + view_version_id: int = Field( + ..., + alias='view-version-id', + description='The view version id to set as current, or -1 to set last added view version id', + ) + + class TableRequirement(BaseModel): type: str @@ -675,6 +718,17 @@ class TableMetadata(BaseModel): metadata_log: Optional[MetadataLog] = Field(None, alias='metadata-log') +class ViewMetadata(BaseModel): + view_uuid: str = Field(..., alias='view-uuid') + format_version: int = Field(..., alias='format-version', ge=1, le=1) + location: str + current_version_id: int = Field(..., alias='current-version-id') + versions: List[ViewVersion] + version_log: List[ViewHistoryEntry] = Field(..., alias='version-log') + schemas: List[Schema] + properties: Optional[Dict[str, str]] = None + + class AddSchemaUpdate(BaseUpdate): schema_: Schema = Field(..., alias='schema') last_column_id: Optional[int] = Field( @@ -704,6 +758,19 @@ class TableUpdate(BaseModel): ] +class ViewUpdate(BaseModel): + __root__: Union[ + AssignUUIDUpdate, + UpgradeFormatVersionUpdate, + AddSchemaUpdate, + SetLocationUpdate, + SetPropertiesUpdate, + RemovePropertiesUpdate, + AddViewVersionUpdate, + SetCurrentViewVersionUpdate, + ] + + class LoadTableResult(BaseModel): """ Result used when a table is successfully loaded. @@ -751,6 +818,13 @@ class CommitTableRequest(BaseModel): updates: List[TableUpdate] +class CommitViewRequest(BaseModel): + identifier: Optional[TableIdentifier] = Field( + None, description='View identifier to update' + ) + updates: List[ViewUpdate] + + class CommitTransactionRequest(BaseModel): table_changes: List[CommitTableRequest] = Field(..., alias='table-changes') @@ -765,6 +839,41 @@ class CreateTableRequest(BaseModel): properties: Optional[Dict[str, str]] = None +class CreateViewRequest(BaseModel): + name: str + location: Optional[str] = None + schema_: Schema = Field(..., alias='schema') + view_version: ViewVersion = Field( + ..., + alias='view-version', + description='The view version to create, will replace the schema-id sent within the view-version with the id assigned to the provided schema', + ) + properties: Dict[str, str] + + +class LoadViewResult(BaseModel): + """ + Result used when a view is successfully loaded. + + + The view metadata JSON is returned in the `metadata` field. The corresponding file location of view metadata is returned in the `metadata-location` field. + Clients can check whether metadata has changed by comparing metadata locations after the view has been created. + + The `config` map returns view-specific configuration for the view's resources. + + The following configurations should be respected by clients: + + ## General Configurations + + - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled + + """ + + metadata_location: str = Field(..., alias='metadata-location') + metadata: ViewMetadata + config: Optional[Dict[str, str]] = None + + class ReportMetricsRequest2(BaseModel): __root__: Union[ReportMetricsRequest, ReportMetricsRequest1] @@ -801,6 +910,8 @@ class ReportMetricsRequest(ScanReport): MapType.update_forward_refs() Expression.update_forward_refs() TableMetadata.update_forward_refs() +ViewMetadata.update_forward_refs() AddSchemaUpdate.update_forward_refs() CreateTableRequest.update_forward_refs() +CreateViewRequest.update_forward_refs() ReportMetricsRequest2.update_forward_refs() diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 9370ef8f2dca..c965793cb229 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -884,7 +884,7 @@ paths: 406: $ref: '#/components/responses/UnsupportedOperationResponse' 409: - description: Conflict - The target table identifier to rename to already exists + description: Conflict - The target identifier to rename to already exists as a table or view content: application/json: schema: @@ -1056,6 +1056,356 @@ paths: } } + /v1/{prefix}/namespaces/{namespace}/views: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + + get: + tags: + - Catalog API + summary: List all view identifiers underneath a given namespace + description: Return all view identifiers under this namespace + operationId: listViews + responses: + 200: + $ref: '#/components/responses/ListTablesResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: Not Found - The namespace specified does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + NamespaceNotFound: + $ref: '#/components/examples/NoSuchNamespaceError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + post: + tags: + - Catalog API + summary: Create a view in the given namespace + description: + Create a view in the given namespace. + operationId: createView + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/CreateViewRequest' + responses: + 200: + $ref: '#/components/responses/LoadViewResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: Not Found - The namespace specified does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + NamespaceNotFound: + $ref: '#/components/examples/NoSuchNamespaceError' + 409: + description: Conflict - The view already exists + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + NamespaceAlreadyExists: + $ref: '#/components/examples/ViewAlreadyExistsError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + /v1/{prefix}/namespaces/{namespace}/views/{view}: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/view' + + get: + tags: + - Catalog API + summary: Load a view from the catalog + operationId: loadView + description: + Load a view from the catalog. + + + The response contains both configuration and view metadata. The configuration, if non-empty is used + as additional configuration for the view that overrides catalog configuration. + + + The response also contains the view's full metadata, matching the view metadata JSON file. + + + The catalog configuration may contain credentials that should be used for subsequent requests for the + view. The configuration key "token" is used to pass an access token to be used as a bearer token + for view requests. Otherwise, a token may be passed using a RFC 8693 token type as a configuration + key. For example, "urn:ietf:params:oauth:token-type:jwt=". + responses: + 200: + $ref: '#/components/responses/LoadViewResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found - NoSuchViewException, view to load does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + ViewToLoadDoesNotExist: + $ref: '#/components/examples/NoSuchViewError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + post: + tags: + - Catalog API + summary: Replace a view + operationId: replaceView + description: + Commit updates to a view. + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/CommitViewRequest' + responses: + 200: + $ref: '#/components/responses/LoadViewResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found - NoSuchViewException, view to load does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + ViewToUpdateDoesNotExist: + $ref: '#/components/examples/NoSuchViewError' + 409: + description: + Conflict - CommitFailedException. The client may retry. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 500: + description: + An unknown server-side problem occurred; the commit state is unknown. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + example: { + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 + } + } + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 502: + description: + A gateway or proxy received an invalid response from the upstream server; the commit state is unknown. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + example: { + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 + } + } + 504: + description: + A server-side gateway timeout occurred; the commit state is unknown. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + example: { + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 + } + } + 5XX: + description: + A server-side problem that might not be addressable on the client. + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + example: { + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 + } + } + + delete: + tags: + - Catalog API + summary: Drop a view from the catalog + operationId: dropView + description: Remove a view from the catalog + responses: + 204: + description: Success, no content + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found - NoSuchViewException, view to drop does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + ViewToDeleteDoesNotExist: + $ref: '#/components/examples/NoSuchViewError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + head: + tags: + - Catalog API + summary: Check if a view exists + operationId: viewExists + description: + Check if a view exists within a given namespace. This request does not return a response body. + responses: + 204: + description: Success, no content + 400: + description: Bad Request + 401: + description: Unauthorized + 404: + description: Not Found + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + /v1/{prefix}/views/rename: + parameters: + - $ref: '#/components/parameters/prefix' + + post: + tags: + - Catalog API + summary: Rename a view from its current name to a new name + description: + Rename a view from one identifier to another. It's valid to move a view + across namespaces, but the server implementation is not required to support it. + operationId: renameView + requestBody: + description: Current view identifier to rename and new view identifier to rename to + content: + application/json: + schema: + $ref: '#/components/schemas/RenameTableRequest' + examples: + RenameViewSameNamespace: + $ref: '#/components/examples/RenameViewSameNamespace' + required: true + responses: + 200: + description: OK + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found + - NoSuchViewException, view to rename does not exist + - NoSuchNamespaceException, The target namespace of the new identifier does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + examples: + ViewToRenameDoesNotExist: + $ref: '#/components/examples/NoSuchViewError' + NamespaceToRenameToDoesNotExist: + $ref: '#/components/examples/NoSuchNamespaceError' + 406: + $ref: '#/components/responses/UnsupportedOperationResponse' + 409: + description: Conflict - The target identifier to rename to already exists as a table or view + content: + application/json: + schema: + $ref: '#/components/schemas/ErrorModel' + example: + $ref: '#/components/examples/ViewAlreadyExistsError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + components: ####################################################### @@ -1094,6 +1444,15 @@ components: type: string example: "sales" + view: + name: view + in: path + description: A view name + required: true + schema: + type: string + example: "sales" + ############################## # Application Schema Objects # ############################## @@ -1671,6 +2030,105 @@ components: metadata-log: $ref: '#/components/schemas/MetadataLog' + SQLViewRepresentation: + type: object + required: + - type + - sql + - dialect + properties: + type: + type: string + sql: + type: string + dialect: + type: string + + ViewRepresentation: + oneOf: + - $ref: '#/components/schemas/SQLViewRepresentation' + + ViewHistoryEntry: + type: object + required: + - version-id + - timestamp-ms + properties: + version-id: + type: integer + timestamp-ms: + type: integer + format: int64 + + ViewVersion: + type: object + required: + - version-id + - timestamp-ms + - schema-id + - summary + - representations + - default-namespace + properties: + version-id: + type: integer + timestamp-ms: + type: integer + format: int64 + schema-id: + type: integer + description: Schema ID to set as current, or -1 to set last added schema + summary: + type: object + additionalProperties: + type: string + representations: + type: array + items: + $ref: '#/components/schemas/ViewRepresentation' + default-catalog: + type: string + default-namespace: + $ref: '#/components/schemas/Namespace' + + ViewMetadata: + type: object + required: + - view-uuid + - format-version + - location + - current-version-id + - versions + - version-log + - schemas + properties: + view-uuid: + type: string + format-version: + type: integer + minimum: 1 + maximum: 1 + location: + type: string + current-version-id: + type: integer + versions: + type: array + items: + $ref: '#/components/schemas/ViewVersion' + version-log: + type: array + items: + $ref: '#/components/schemas/ViewHistoryEntry' + schemas: + type: array + items: + $ref: '#/components/schemas/Schema' + properties: + type: object + additionalProperties: + type: string + BaseUpdate: type: object required: @@ -1694,6 +2152,8 @@ components: - set-location - set-properties - remove-properties + - add-view-version + - set-current-view-version AssignUUIDUpdate: description: Assigning a UUID to a table/view should only be done when creating the table/view. It is not safe to re-assign the UUID if a table/view already has a UUID assigned @@ -1860,6 +2320,27 @@ components: items: type: string + AddViewVersionUpdate: + allOf: + - $ref: '#/components/schemas/BaseUpdate' + - type: object + required: + - view-version + properties: + view-version: + $ref: '#/components/schemas/ViewVersion' + + SetCurrentViewVersionUpdate: + allOf: + - $ref: '#/components/schemas/BaseUpdate' + - type: object + required: + - view-version-id + properties: + view-version-id: + type: integer + description: The view version id to set as current, or -1 to set last added view version id + TableUpdate: anyOf: - $ref: '#/components/schemas/AssignUUIDUpdate' @@ -1878,6 +2359,17 @@ components: - $ref: '#/components/schemas/SetPropertiesUpdate' - $ref: '#/components/schemas/RemovePropertiesUpdate' + ViewUpdate: + anyOf: + - $ref: '#/components/schemas/AssignUUIDUpdate' + - $ref: '#/components/schemas/UpgradeFormatVersionUpdate' + - $ref: '#/components/schemas/AddSchemaUpdate' + - $ref: '#/components/schemas/SetLocationUpdate' + - $ref: '#/components/schemas/SetPropertiesUpdate' + - $ref: '#/components/schemas/RemovePropertiesUpdate' + - $ref: '#/components/schemas/AddViewVersionUpdate' + - $ref: '#/components/schemas/SetCurrentViewVersionUpdate' + TableRequirement: discriminator: propertyName: type @@ -2076,6 +2568,19 @@ components: items: $ref: '#/components/schemas/TableUpdate' + CommitViewRequest: + type: object + required: + - updates + properties: + identifier: + description: View identifier to update + $ref: '#/components/schemas/TableIdentifier' + updates: + type: array + items: + $ref: '#/components/schemas/ViewUpdate' + CommitTransactionRequest: type: object required: @@ -2121,6 +2626,58 @@ components: metadata-location: type: string + CreateViewRequest: + type: object + required: + - name + - schema + - view-version + - properties + properties: + name: + type: string + location: + type: string + schema: + $ref: '#/components/schemas/Schema' + view-version: + $ref: '#/components/schemas/ViewVersion' + description: The view version to create, will replace the schema-id sent within the view-version with the id assigned to the provided schema + properties: + type: object + additionalProperties: + type: string + + LoadViewResult: + description: | + Result used when a view is successfully loaded. + + + The view metadata JSON is returned in the `metadata` field. The corresponding file location of view metadata is returned in the `metadata-location` field. + Clients can check whether metadata has changed by comparing metadata locations after the view has been created. + + The `config` map returns view-specific configuration for the view's resources. + + The following configurations should be respected by clients: + + ## General Configurations + + - `token`: Authorization bearer token to use for view requests if OAuth2 security is enabled + + type: object + required: + - metadata-location + - metadata + properties: + metadata-location: + type: string + metadata: + $ref: '#/components/schemas/ViewMetadata' + config: + type: object + additionalProperties: + type: string + TokenType: type: string enum: @@ -2742,6 +3299,13 @@ components: schema: $ref: '#/components/schemas/LoadTableResult' + LoadViewResponse: + description: View metadata result when loading a view + content: + application/json: + schema: + $ref: '#/components/schemas/LoadViewResult' + CommitTableResponse: description: Response used when a table is successfully updated. @@ -2806,7 +3370,7 @@ components: } NoSuchTableError: - summary: The requested table does not + summary: The requested table does not exist value: { "error": { "message": "The given table does not exist", @@ -2815,6 +3379,16 @@ components: } } + NoSuchViewError: + summary: The requested view does not exist + value: { + "error": { + "message": "The given view does not exist", + "type": "NoSuchViewException", + "code": 404 + } + } + NoSuchNamespaceError: summary: The requested namespace does not exist value: { @@ -2832,6 +3406,13 @@ components: "destination": { "namespace": ["accounting", "tax"], "name": "owed" } } + RenameViewSameNamespace: + summary: Rename a view in the same namespace + value: { + "source": { "namespace": [ "accounting", "tax" ], "name": "paid-view" }, + "destination": { "namespace": [ "accounting", "tax" ], "name": "owed-view" } + } + TableAlreadyExistsError: summary: The requested table identifier already exists value: { @@ -2842,6 +3423,16 @@ components: } } + ViewAlreadyExistsError: + summary: The requested view identifier already exists + value: { + "error": { + "message": "The given view already exists", + "type": "AlreadyExistsException", + "code": 409 + } + } + # This is an example response and is not meant to be prescriptive regarding the message or type. UnprocessableEntityDuplicateKey: summary: From afe4aec4db795f4829757d738a5bbcf1b7db8fd2 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 5 Dec 2023 17:14:09 +0100 Subject: [PATCH 84/94] Spark: Don't allow branch_ usage with VERSION AS OF (#9219) --- .../org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../apache/iceberg/spark/source/SparkTable.java | 4 ++++ .../org/apache/iceberg/spark/sql/TestSelect.java | 15 +++++++++++++++ .../org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../apache/iceberg/spark/source/SparkTable.java | 4 ++++ .../org/apache/iceberg/spark/sql/TestSelect.java | 15 +++++++++++++++ .../org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../apache/iceberg/spark/source/SparkTable.java | 4 ++++ .../org/apache/iceberg/spark/sql/TestSelect.java | 15 +++++++++++++++ 9 files changed, 60 insertions(+), 3 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 02bbec6824f3..48decf995fbb 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -171,7 +171,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); try { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index 574d014e8335..eddcdb1819ee 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -173,6 +173,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index e08bc4574dbf..1368c26792ee 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -426,6 +426,21 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { }); } + @Test + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + @Test public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 6958ebc1034a..6b7becc77cd7 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -170,7 +170,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); try { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index e200bee03e9f..bbc7434138ed 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -173,6 +173,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index dacaee7d8030..161c2e0ba637 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -421,6 +421,21 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } + @Test + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + @Test public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 467b42899480..eef0f0703bc3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -170,7 +170,7 @@ public Table loadTable(Identifier ident, String version) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); try { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java index e200bee03e9f..bbc7434138ed 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkTable.java @@ -173,6 +173,10 @@ public Long snapshotId() { return snapshotId; } + public String branch() { + return branch; + } + public SparkTable copyWithSnapshotId(long newSnapshotId) { return new SparkTable(icebergTable, newSnapshotId, refreshEagerly); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index dacaee7d8030..161c2e0ba637 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -421,6 +421,21 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } + @Test + public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { + long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + validationCatalog.loadTable(tableIdent).manageSnapshots().createBranch("b1").commit(); + + // create a second snapshot + sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); + + // using branch_b1 in the table identifier and VERSION AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + } + @Test public void testSpecifySnapshotAndTimestamp() { // get the snapshot ID of the last write From 8519224de33b26cfd7c539ffe6f123ea66165711 Mon Sep 17 00:00:00 2001 From: pvary Date: Tue, 5 Dec 2023 17:48:10 +0100 Subject: [PATCH 85/94] Flink: Document watermark generation feature (#9179) --- docs/flink-queries.md | 69 +++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 69 insertions(+) diff --git a/docs/flink-queries.md b/docs/flink-queries.md index 4cef5468cd1a..cf68fa367c21 100644 --- a/docs/flink-queries.md +++ b/docs/flink-queries.md @@ -277,6 +277,75 @@ DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks() "Iceberg Source as Avro GenericRecord", new GenericRecordAvroTypeInfo(avroSchema)); ``` +### Emitting watermarks +Emitting watermarks from the source itself could be beneficial for several purposes, like harnessing the +[Flink Watermark Alignment](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/datastream/event-time/generating_watermarks/#watermark-alignment), +or prevent triggering [windows](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/datastream/operators/windows/) +too early when reading multiple data files concurrently. + +Enable watermark generation for an `IcebergSource` by setting the `watermarkColumn`. +The supported column types are `timestamp`, `timestamptz` and `long`. +Iceberg `timestamp` or `timestamptz` inherently contains the time precision. So there is no need +to specify the time unit. But `long` type column doesn't contain time unit information. Use +`watermarkTimeUnit` to configure the conversion for long columns. + +The watermarks are generated based on column metrics stored for data files and emitted once per split. +If multiple smaller files with different time ranges are combined into a single split, it can increase +the out-of-orderliness and extra data buffering in the Flink state. The main purpose of watermark alignment +is to reduce out-of-orderliness and excess data buffering in the Flink state. Hence it is recommended to +set `read.split.open-file-cost` to a very large value to prevent combining multiple smaller files into a +single split. The negative impact (of not combining small files into a single split) is on read throughput, +especially if there are many small files. In typical stateful processing jobs, source read throughput is not +the bottleneck. Hence this is probably a reasonable tradeoff. + +This feature requires column-level min-max stats. Make sure stats are generated for the watermark column +during write phase. By default, the column metrics are collected for the first 100 columns of the table. +If watermark column doesn't have stats enabled by default, use +[write properties](configuration.md#write-properties) starting with `write.metadata.metrics` when needed. + +The following example could be useful if watermarks are used for windowing. The source reads Iceberg data files +in order, using a timestamp column and emits watermarks: +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); +TableLoader tableLoader = TableLoader.fromHadoopTable("hdfs://nn:8020/warehouse/path"); + +DataStream stream = + env.fromSource( + IcebergSource.forRowData() + .tableLoader(tableLoader) + // Watermark using timestamp column + .watermarkColumn("timestamp_column") + .build(), + // Watermarks are generated by the source, no need to generate it manually + WatermarkStrategy.noWatermarks() + // Extract event timestamp from records + .withTimestampAssigner((record, eventTime) -> record.getTimestamp(pos, precision).getMillisecond()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); +``` + +Example for reading Iceberg table using a long event column for watermark alignment: +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(); +TableLoader tableLoader = TableLoader.fromHadoopTable("hdfs://nn:8020/warehouse/path"); + +DataStream stream = + env.fromSource( + IcebergSource source = IcebergSource.forRowData() + .tableLoader(tableLoader) + // Disable combining multiple files to a single split + .set(FlinkReadOptions.SPLIT_FILE_OPEN_COST, String.valueOf(TableProperties.SPLIT_SIZE_DEFAULT)) + // Watermark using long column + .watermarkColumn("long_column") + .watermarkTimeUnit(TimeUnit.MILLI_SCALE) + .build(), + // Watermarks are generated by the source, no need to generate it manually + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment(watermarkGroup, maxAllowedWatermarkDrift), + SOURCE_NAME, + TypeInformation.of(RowData.class)); +``` + ## Options ### Read options From 68d491e6a344a0e081cc784db28dec60670d4d36 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 5 Dec 2023 18:27:27 +0100 Subject: [PATCH 86/94] Build: Bump datamodel-code-generator from 0.24.2 to 0.25.0 (#9189) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.24.2 to 0.25.0. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.24.2...0.25.0) --- updated-dependencies: - dependency-name: datamodel-code-generator dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- open-api/requirements.txt | 2 +- open-api/rest-catalog-open-api.py | 10 +++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/open-api/requirements.txt b/open-api/requirements.txt index a9be79a486fb..c20e40f4ad2a 100644 --- a/open-api/requirements.txt +++ b/open-api/requirements.txt @@ -16,4 +16,4 @@ # under the License. openapi-spec-validator==0.7.1 -datamodel-code-generator==0.24.2 +datamodel-code-generator==0.25.0 diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 5da91a16f901..5a17b9d43856 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -614,7 +614,7 @@ class TransformTerm(BaseModel): term: Reference -class ReportMetricsRequest1(CommitReport): +class ReportMetricsRequest2(CommitReport): report_type: str = Field(..., alias='report-type') @@ -874,8 +874,8 @@ class LoadViewResult(BaseModel): config: Optional[Dict[str, str]] = None -class ReportMetricsRequest2(BaseModel): - __root__: Union[ReportMetricsRequest, ReportMetricsRequest1] +class ReportMetricsRequest(BaseModel): + __root__: Union[ReportMetricsRequest1, ReportMetricsRequest2] class ScanReport(BaseModel): @@ -901,7 +901,7 @@ class Schema(StructType): ) -class ReportMetricsRequest(ScanReport): +class ReportMetricsRequest1(ScanReport): report_type: str = Field(..., alias='report-type') @@ -914,4 +914,4 @@ class ReportMetricsRequest(ScanReport): AddSchemaUpdate.update_forward_refs() CreateTableRequest.update_forward_refs() CreateViewRequest.update_forward_refs() -ReportMetricsRequest2.update_forward_refs() +ReportMetricsRequest.update_forward_refs() From 7b12a4171e9e6b7829e7a66b447fc15cada151fc Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Tue, 5 Dec 2023 09:44:06 -0800 Subject: [PATCH 87/94] Flink: backport PR #9216 for disabling classloader check (#9226) --- .../source/TestIcebergSourceWithWatermarkExtractor.java | 6 +++++- .../source/TestIcebergSourceWithWatermarkExtractor.java | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 7547323871c1..ac4d07fa52d3 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -42,6 +42,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; @@ -98,7 +99,10 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(new Configuration())) + .setConfiguration( + reporter.addToConfiguration( + // disable classloader check as Avro may cache class in the serializers. + new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) .withHaLeadershipControl() .build()); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 0bb2eb7766e9..aa4b9cd79e55 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -40,6 +40,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; @@ -92,7 +93,10 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(new Configuration())) + .setConfiguration( + reporter.addToConfiguration( + // disable classloader check as Avro may cache class in the serializers. + new Configuration().set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false))) .withHaLeadershipControl() .build()); From 8b7a280a9fd0b51eb43538fbb9f6879d1c92de18 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 5 Dec 2023 18:45:02 +0100 Subject: [PATCH 88/94] Build: Bump actions/setup-java from 3 to 4 (#9200) Bumps [actions/setup-java](https://github.com/actions/setup-java) from 3 to 4. - [Release notes](https://github.com/actions/setup-java/releases) - [Commits](https://github.com/actions/setup-java/compare/v3...v4) --- updated-dependencies: - dependency-name: actions/setup-java dependency-type: direct:production update-type: version-update:semver-major ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .github/workflows/api-binary-compatibility.yml | 2 +- .github/workflows/delta-conversion-ci.yml | 4 ++-- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 4 ++-- .github/workflows/java-ci.yml | 6 +++--- .github/workflows/jmh-benchmarks.yml | 2 +- .github/workflows/publish-snapshot.yml | 2 +- .github/workflows/recurring-jmh-benchmarks.yml | 2 +- .github/workflows/spark-ci.yml | 6 +++--- 9 files changed, 15 insertions(+), 15 deletions(-) diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index fd7f6f14c2ef..245da4ecd4d1 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -46,7 +46,7 @@ jobs: # # See https://github.com/actions/checkout/issues/124 fetch-depth: 0 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 11 diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index 67f5acdb32b2..5261b8217624 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -62,7 +62,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -91,7 +91,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 4ea17af6c0cb..30fe6c7723bc 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -65,7 +65,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index d7474a185436..f582e516fcd1 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -60,7 +60,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -86,7 +86,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 4c8e3a5b8d92..4936e2b6514b 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -57,7 +57,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -81,7 +81,7 @@ jobs: runs-on: ubuntu-22.04 steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 @@ -91,7 +91,7 @@ jobs: runs-on: ubuntu-22.04 steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 diff --git a/.github/workflows/jmh-benchmarks.yml b/.github/workflows/jmh-benchmarks.yml index f2d6c659f61e..e76590543852 100644 --- a/.github/workflows/jmh-benchmarks.yml +++ b/.github/workflows/jmh-benchmarks.yml @@ -79,7 +79,7 @@ jobs: with: repository: ${{ github.event.inputs.repo }} ref: ${{ github.event.inputs.ref }} - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 11 diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 92c6b2b09223..c2809cfdcbac 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -34,7 +34,7 @@ jobs: with: # we need to fetch all tags so that getProjectVersion() in build.gradle correctly determines the next SNAPSHOT version from the newest tag fetch-depth: 0 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 8 diff --git a/.github/workflows/recurring-jmh-benchmarks.yml b/.github/workflows/recurring-jmh-benchmarks.yml index e1f750041890..434b79755069 100644 --- a/.github/workflows/recurring-jmh-benchmarks.yml +++ b/.github/workflows/recurring-jmh-benchmarks.yml @@ -49,7 +49,7 @@ jobs: with: repository: ${{ github.event.inputs.repo }} ref: ${{ github.event.inputs.ref }} - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 11 diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index a05c9aa19e29..3dacdf77c1a8 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -63,7 +63,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -93,7 +93,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: ${{ matrix.jvm }} @@ -123,7 +123,7 @@ jobs: SPARK_LOCAL_IP: localhost steps: - uses: actions/checkout@v4 - - uses: actions/setup-java@v3 + - uses: actions/setup-java@v4 with: distribution: zulu java-version: 17 From d80d7da3d0a956e1b6aeadbe26e0cb3f3ef2fb25 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 5 Dec 2023 19:04:47 +0100 Subject: [PATCH 89/94] Core: Handle IAE in default error handler (#9225) --- core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java | 3 +++ .../test/java/org/apache/iceberg/view/ViewCatalogTests.java | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java index 846820a99d9f..d4083420efa6 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/ErrorHandlers.java @@ -202,6 +202,9 @@ public ErrorResponse parseResponse(int code, String json) { public void accept(ErrorResponse error) { switch (error.code()) { case 400: + if (IllegalArgumentException.class.getSimpleName().equals(error.type())) { + throw new IllegalArgumentException(error.message()); + } throw new BadRequestException("Malformed request: %s", error.message()); case 401: throw new NotAuthorizedException("Not authorized: %s", error.message()); diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 8cb77a776220..8280a200d605 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -252,7 +252,7 @@ public void createViewErrorCases() { .withQuery(trino.dialect(), trino.sql()) .withQuery(trino.dialect(), trino.sql()) .create()) - .isInstanceOf(Exception.class) + .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Invalid view version: Cannot add multiple queries for dialect trino"); } From faa8b5075cb70d1cebea54700e39f038c623f08e Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 5 Dec 2023 13:13:33 -0800 Subject: [PATCH 90/94] Core: Fix logic for determining set of committed files in BaseTransaction when there are no new snapshots (#9221) --- .../org/apache/iceberg/BaseTransaction.java | 29 ++++++++++++------- 1 file changed, 18 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index 018f70eb16fa..30103fd87fe2 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -45,6 +45,7 @@ import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.PropertyUtil; @@ -446,16 +447,20 @@ private void commitSimpleTransaction() { } Set committedFiles = committedFiles(ops, newSnapshots); - // delete all of the files that were deleted in the most recent set of operation commits - Tasks.foreach(deletedFiles) - .suppressFailureWhenFinished() - .onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc)) - .run( - path -> { - if (committedFiles == null || !committedFiles.contains(path)) { - ops.io().deleteFile(path); - } - }); + if (committedFiles != null) { + // delete all of the files that were deleted in the most recent set of operation commits + Tasks.foreach(deletedFiles) + .suppressFailureWhenFinished() + .onFailure((file, exc) -> LOG.warn("Failed to delete uncommitted file: {}", file, exc)) + .run( + path -> { + if (!committedFiles.contains(path)) { + ops.io().deleteFile(path); + } + }); + } else { + LOG.warn("Failed to load metadata for a committed snapshot, skipping clean-up"); + } } catch (RuntimeException e) { LOG.warn("Failed to load committed metadata, skipping clean-up", e); @@ -502,9 +507,11 @@ private void applyUpdates(TableOperations underlyingOps) { } } + // committedFiles returns null whenever the set of committed files + // cannot be determined from the provided snapshots private static Set committedFiles(TableOperations ops, Set snapshotIds) { if (snapshotIds.isEmpty()) { - return null; + return ImmutableSet.of(); } Set committedFiles = Sets.newHashSet(); From 8e1900dc9ab917cd97a41e7145166a8c8a4bdfc5 Mon Sep 17 00:00:00 2001 From: Junhao Liu Date: Tue, 5 Dec 2023 15:48:30 -0600 Subject: [PATCH 91/94] Style: Replace Arrays.asList with Collections.singletonList (#9213) --- .../aws/s3/signer/TestS3SignRequestParser.java | 13 +++++++------ .../aws/s3/signer/TestS3SignResponseParser.java | 5 +++-- .../org/apache/iceberg/BaseRewriteManifests.java | 4 ++-- .../test/java/org/apache/iceberg/ScanTestBase.java | 12 ++++++------ .../org/apache/iceberg/TestContentFileParser.java | 3 ++- .../enumerator/TestContinuousIcebergEnumerator.java | 11 ++++++----- .../source/reader/TestIcebergSourceReader.java | 2 +- .../source/reader/TestIcebergSourceReader.java | 2 +- .../source/reader/TestIcebergSourceReader.java | 2 +- .../org/apache/iceberg/hive/TestHiveSchemaUtil.java | 7 +++++-- 10 files changed, 34 insertions(+), 27 deletions(-) diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java index a6928183f705..2c5f74ad8064 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import java.net.URI; import java.util.Arrays; +import java.util.Collections; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -122,9 +123,9 @@ public void roundTripSerde() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .build(); @@ -158,9 +159,9 @@ public void roundTripSerdeWithProperties() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .properties(ImmutableMap.of("k1", "v1")) @@ -198,9 +199,9 @@ public void roundTripWithBody() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .properties(ImmutableMap.of("k1", "v1")) diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java index d7337b1b1777..d2cf132ba598 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import java.net.URI; import java.util.Arrays; +import java.util.Collections; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -70,9 +71,9 @@ public void roundTripSerde() { "amz-sdk-request", Arrays.asList("attempt=1", "max=4"), "Content-Length", - Arrays.asList("191"), + Collections.singletonList("191"), "Content-Type", - Arrays.asList("application/json"), + Collections.singletonList("application/json"), "User-Agent", Arrays.asList("aws-sdk-java/2.20.18", "Linux/5.4.0-126"))) .build(); diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index 87768e34894a..c70dda2bd6d0 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -22,8 +22,8 @@ import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT; import java.io.IOException; -import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -247,7 +247,7 @@ private void performRewrite(List currentManifests) { rewrittenManifests.add(manifest); try (ManifestReader reader = ManifestFiles.read(manifest, ops.io(), ops.current().specsById()) - .select(Arrays.asList("*"))) { + .select(Collections.singletonList("*"))) { reader .liveEntries() .forEach( diff --git a/core/src/test/java/org/apache/iceberg/ScanTestBase.java b/core/src/test/java/org/apache/iceberg/ScanTestBase.java index 5e7793939925..48a8ccbaa941 100644 --- a/core/src/test/java/org/apache/iceberg/ScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanTestBase.java @@ -23,7 +23,7 @@ import java.io.File; import java.io.IOException; -import java.util.Arrays; +import java.util.Collections; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.expressions.Expression; @@ -56,7 +56,7 @@ public ScanTestBase(int formatVersion) { @Test public void testTableScanHonorsSelect() { - ScanT scan = newScan().select(Arrays.asList("id")); + ScanT scan = newScan().select(Collections.singletonList("id")); Schema expectedSchema = new Schema(required(1, "id", Types.IntegerType.get())); @@ -69,20 +69,20 @@ public void testTableScanHonorsSelect() { @Test public void testTableBothProjectAndSelect() { Assertions.assertThatThrownBy( - () -> newScan().select(Arrays.asList("id")).project(SCHEMA.select("data"))) + () -> newScan().select(Collections.singletonList("id")).project(SCHEMA.select("data"))) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot set projection schema when columns are selected"); Assertions.assertThatThrownBy( - () -> newScan().project(SCHEMA.select("data")).select(Arrays.asList("id"))) + () -> newScan().project(SCHEMA.select("data")).select(Collections.singletonList("id"))) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot select columns when projection schema is set"); } @Test public void testTableScanHonorsSelectWithoutCaseSensitivity() { - ScanT scan1 = newScan().caseSensitive(false).select(Arrays.asList("ID")); + ScanT scan1 = newScan().caseSensitive(false).select(Collections.singletonList("ID")); // order of refinements shouldn't matter - ScanT scan2 = newScan().select(Arrays.asList("ID")).caseSensitive(false); + ScanT scan2 = newScan().select(Collections.singletonList("ID")).caseSensitive(false); Schema expectedSchema = new Schema(required(1, "id", Types.IntegerType.get())); diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index f3b5e7b60c31..4fda388159ec 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.JsonNode; import java.nio.ByteBuffer; import java.util.Arrays; +import java.util.Collections; import java.util.stream.Stream; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Comparators; @@ -270,7 +271,7 @@ private static DeleteFile deleteFileWithAllOptional(PartitionSpec spec) { metrics, new int[] {3}, 1, - Arrays.asList(128L), + Collections.singletonList(128L), ByteBuffer.wrap(new byte[16])); } diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index 349eb11cf549..5b0ed39745c5 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.source.enumerator; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -173,7 +172,7 @@ public void testThrottlingDiscovery() throws Exception { enumerator.handleSourceEvent(2, new SplitRequestEvent()); // add splits[0] to the planner for next discovery - splitPlanner.addSplits(Arrays.asList(splits.get(0))); + splitPlanner.addSplits(Collections.singletonList(splits.get(0))); enumeratorContext.triggerAllActions(); // because discovered split was assigned to reader, pending splits should be empty @@ -185,7 +184,7 @@ public void testThrottlingDiscovery() throws Exception { // add the remaining 9 splits (one for every snapshot) // run discovery cycles while reader-2 still processing the splits[0] for (int i = 1; i < 10; ++i) { - splitPlanner.addSplits(Arrays.asList(splits.get(i))); + splitPlanner.addSplits(Collections.singletonList(splits.get(i))); enumeratorContext.triggerAllActions(); } @@ -196,7 +195,8 @@ public void testThrottlingDiscovery() throws Exception { splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); // now reader-2 finished splits[0] - enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); + enumerator.handleSourceEvent( + 2, new SplitRequestEvent(Collections.singletonList(splits.get(0).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was // discovered and added. @@ -217,7 +217,8 @@ public void testThrottlingDiscovery() throws Exception { splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); // now reader-2 finished splits[1] - enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); + enumerator.handleSourceEvent( + 2, new SplitRequestEvent(Collections.singletonList(splits.get(1).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was // discovered and added. diff --git a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 88234c61123f..f19d57083b89 100644 --- a/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.15/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -132,7 +132,7 @@ private void testOneSplitFetcher( ReaderUtil.createCombinedScanTask( recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Arrays.asList(split)); + reader.addSplits(Collections.singletonList(split)); while (readerOutput.getEmittedRecords().size() < expectedCount) { reader.pollNext(readerOutput); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 88234c61123f..f19d57083b89 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -132,7 +132,7 @@ private void testOneSplitFetcher( ReaderUtil.createCombinedScanTask( recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Arrays.asList(split)); + reader.addSplits(Collections.singletonList(split)); while (readerOutput.getEmittedRecords().size() < expectedCount) { reader.pollNext(readerOutput); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 88234c61123f..f19d57083b89 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -132,7 +132,7 @@ private void testOneSplitFetcher( ReaderUtil.createCombinedScanTask( recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Arrays.asList(split)); + reader.addSplits(Collections.singletonList(split)); while (readerOutput.getEmittedRecords().size() < expectedCount) { reader.pollNext(readerOutput); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java index 84d11d03a741..1592a3461b40 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveSchemaUtil.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.hive.metastore.api.FieldSchema; @@ -147,7 +148,9 @@ public void testSchemaConvertToIcebergSchemaForEveryPrimitiveType() { public void testNotSupportedTypes() { for (FieldSchema notSupportedField : getNotSupportedFieldSchemas()) { assertThatThrownBy( - () -> HiveSchemaUtil.convert(Lists.newArrayList(Arrays.asList(notSupportedField)))) + () -> + HiveSchemaUtil.convert( + Lists.newArrayList(Collections.singletonList(notSupportedField)))) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Unsupported Hive type"); } @@ -197,7 +200,7 @@ public void testConversionWithoutLastComment() { Arrays.asList( TypeInfoUtils.getTypeInfoFromTypeString(serdeConstants.BIGINT_TYPE_NAME), TypeInfoUtils.getTypeInfoFromTypeString(serdeConstants.STRING_TYPE_NAME)), - Arrays.asList("customer comment")); + Collections.singletonList("customer comment")); assertThat(schema.asStruct()).isEqualTo(expected.asStruct()); } From 367dc8b1b94fd8cbae2b5e4da5d01225e717c49b Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 5 Dec 2023 17:27:42 -0800 Subject: [PATCH 92/94] Core: Add comment property to ViewProperties (#9181) --- .../main/java/org/apache/iceberg/view/ViewProperties.java | 1 + .../org/apache/iceberg/view/TestViewMetadataParser.java | 7 +++++-- .../org/apache/iceberg/view/ValidViewMetadata.json | 2 +- 3 files changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/view/ViewProperties.java b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java index b356db7ce040..12b63659bcb2 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewProperties.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java @@ -25,6 +25,7 @@ public class ViewProperties { public static final String METADATA_COMPRESSION = "write.metadata.compression-codec"; public static final String METADATA_COMPRESSION_DEFAULT = "gzip"; + public static final String COMMENT = "comment"; private ViewProperties() {} } diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java index 613b8a7fddb1..d2372363723b 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java @@ -107,7 +107,8 @@ public void readAndWriteValidViewMetadata() throws Exception { .addVersion(version1) .addVersion(version2) .setLocation("s3://bucket/test/location") - .setProperties(ImmutableMap.of("some-key", "some-value")) + .setProperties( + ImmutableMap.of("some-key", "some-value", ViewProperties.COMMENT, "some-comment")) .setCurrentVersionId(2) .upgradeFormatVersion(1) .build(); @@ -218,7 +219,9 @@ public void viewMetadataWithMetadataLocation() throws Exception { .addVersion(version1) .addVersion(version2) .setLocation("s3://bucket/test/location") - .setProperties(ImmutableMap.of("some-key", "some-value")) + .setProperties( + ImmutableMap.of( + "some-key", "some-value", ViewProperties.COMMENT, "some-comment")) .setCurrentVersionId(2) .upgradeFormatVersion(1) .build()) diff --git a/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json b/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json index cf022c5a7098..4b600d6f6b47 100644 --- a/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json +++ b/core/src/test/resources/org/apache/iceberg/view/ValidViewMetadata.json @@ -2,7 +2,7 @@ "view-uuid": "fa6506c3-7681-40c8-86dc-e36561f83385", "format-version": 1, "location": "s3://bucket/test/location", - "properties": {"some-key": "some-value"}, + "properties": {"some-key": "some-value", "comment": "some-comment"}, "current-schema-id": 0, "schemas": [ { From a89fc4646ae5e328a2ccadb98d41b79632afcf3b Mon Sep 17 00:00:00 2001 From: emkornfield Date: Wed, 6 Dec 2023 00:07:46 -0800 Subject: [PATCH 93/94] Spec: Clarify how column IDs are required (#9162) --- format/spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index 5d6dded5ee76..80cdd6d2987f 100644 --- a/format/spec.md +++ b/format/spec.md @@ -963,7 +963,7 @@ Note that the string map case is for maps where the key type is a string. Using **Data Type Mappings** -Values should be stored in Parquet using the types and logical type annotations in the table below. Column IDs are required. +Values should be stored in Parquet using the types and logical type annotations in the table below. Column IDs are required to be stored as [field IDs](http://github.com/apache/parquet-format/blob/40699d05bd24181de6b1457babbee2c16dce3803/src/main/thrift/parquet.thrift#L459) on the parquet schema. Lists must use the [3-level representation](https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists). From 70ec4e5ead8db9bd946c44731192b47862bfdeea Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Wed, 6 Dec 2023 21:06:33 +0530 Subject: [PATCH 94/94] Spark: Bump Spark minor versions for 3.3 and 3.4 (#9187) --- gradle/libs.versions.toml | 4 ++-- .../iceberg/spark/extensions/TestAddFilesProcedure.java | 2 +- .../iceberg/spark/extensions/TestSnapshotTableProcedure.java | 2 +- .../integration/java/org/apache/iceberg/spark/SmokeTest.java | 4 ++-- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index d9ca188e9384..f5c391fa95b5 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -77,8 +77,8 @@ scala-collection-compat = "2.11.0" slf4j = "1.7.36" snowflake-jdbc = "3.14.3" spark-hive32 = "3.2.2" -spark-hive33 = "3.3.2" -spark-hive34 = "3.4.1" +spark-hive33 = "3.3.3" +spark-hive34 = "3.4.2" spark-hive35 = "3.5.0" spring-boot = "2.5.4" spring-web = "5.3.30" diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index db8ffe07b8b7..63396e0969d6 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -77,7 +77,7 @@ public void setupTempDirs() { @After public void dropTables() { - sql("DROP TABLE IF EXISTS %s", sourceTableName); + sql("DROP TABLE IF EXISTS %s PURGE", sourceTableName); sql("DROP TABLE IF EXISTS %s", tableName); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 6e2bf99c5422..367d95d8f619 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -46,7 +46,7 @@ public TestSnapshotTableProcedure( @After public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); - sql("DROP TABLE IF EXISTS %S", sourceName); + sql("DROP TABLE IF EXISTS %s PURGE", sourceName); } @Test diff --git a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index d6fab897d62d..25d7e7471588 100644 --- a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -52,7 +52,7 @@ public void testGettingStarted() throws IOException { Assert.assertEquals( "Should have inserted 3 rows", 3L, scalarSql("SELECT COUNT(*) FROM %s", tableName)); - sql("DROP TABLE IF EXISTS source"); + sql("DROP TABLE IF EXISTS source PURGE"); sql( "CREATE TABLE source (id bigint, data string) USING parquet LOCATION '%s'", temp.newFolder()); @@ -62,7 +62,7 @@ public void testGettingStarted() throws IOException { Assert.assertEquals( "Table should now have 4 rows", 4L, scalarSql("SELECT COUNT(*) FROM %s", tableName)); - sql("DROP TABLE IF EXISTS updates"); + sql("DROP TABLE IF EXISTS updates PURGE"); sql( "CREATE TABLE updates (id bigint, data string) USING parquet LOCATION '%s'", temp.newFolder());