From e603a8face6f9fcb0773880989de66d415708f3f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 1 May 2025 10:53:49 -0400 Subject: [PATCH 01/20] add Iceberg table provider and tests --- sdks/java/extensions/sql/build.gradle | 3 + .../meta/provider/iceberg/IcebergTable.java | 109 +++++++++++ .../iceberg/IcebergTableProvider.java | 34 ++++ .../provider/iceberg/IcebergReadWriteIT.java | 175 ++++++++++++++++++ .../iceberg/IcebergTableProviderTest.java | 95 ++++++++++ 5 files changed, 416 insertions(+) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java create mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java create mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index 6f34891c2d3f..72cae1c2031c 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -74,6 +74,9 @@ dependencies { fmppTask "org.freemarker:freemarker:2.3.31" fmppTemplates library.java.vendored_calcite_1_28_0 implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:managed") + implementation project(":sdks:java:io:iceberg") + implementation project(":sdks:java:io:iceberg:bqms") implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:join-library") permitUnusedDeclared project(":sdks:java:extensions:join-library") // BEAM-11761 diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java new file mode 100644 index 000000000000..8497076637a4 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java @@ -0,0 +1,109 @@ +/* + * 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.beam.sdk.extensions.sql.meta.provider.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.HashMap; +import java.util.Map; +import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.managed.Managed; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.checkerframework.checker.nullness.qual.Nullable; + +class IcebergTable extends SchemaBaseBeamTable { + @VisibleForTesting static final String CATALOG_PROPERTIES_FIELD = "catalog_properties"; + @VisibleForTesting static final String HADOOP_CONFIG_PROPERTIES_FIELD = "config_properties"; + @VisibleForTesting static final String CATALOG_NAME_FIELD = "catalog_name"; + + @VisibleForTesting + static final String TRIGGERING_FREQUENCY_FIELD = "triggering_frequency_seconds"; + + @VisibleForTesting final String tableIdentifier; + @VisibleForTesting @Nullable Map catalogProps; + @VisibleForTesting @Nullable Map configProps; + @VisibleForTesting @Nullable String catalogName; + @VisibleForTesting @Nullable Integer triggeringFrequency; + + IcebergTable(Table table) { + super(table.getSchema()); + this.schema = table.getSchema(); + this.tableIdentifier = checkArgumentNotNull(table.getLocation()); + ObjectNode properties = table.getProperties(); + ObjectMapper objectMapper = new ObjectMapper(); + if (properties.has(CATALOG_PROPERTIES_FIELD)) { + this.catalogProps = + objectMapper.convertValue(properties.get(CATALOG_PROPERTIES_FIELD), HashMap.class); + } + if (properties.has(HADOOP_CONFIG_PROPERTIES_FIELD)) { + this.configProps = + objectMapper.convertValue(properties.get(HADOOP_CONFIG_PROPERTIES_FIELD), HashMap.class); + } + if (properties.has(CATALOG_NAME_FIELD)) { + this.catalogName = properties.get(CATALOG_NAME_FIELD).asText(); + } + if (properties.has(TRIGGERING_FREQUENCY_FIELD)) { + this.triggeringFrequency = properties.get(TRIGGERING_FREQUENCY_FIELD).asInt(); + } + } + + @Override + public PCollection buildIOReader(PBegin begin) { + return begin + .apply(Managed.read(Managed.ICEBERG).withConfig(getBaseConfig())) + .getSinglePCollection(); + } + + @Override + public POutput buildIOWriter(PCollection input) { + ImmutableMap.Builder configBuilder = ImmutableMap.builder(); + configBuilder.putAll(getBaseConfig()); + if (triggeringFrequency != null) { + configBuilder.put(TRIGGERING_FREQUENCY_FIELD, triggeringFrequency); + } + return input.apply(Managed.write(Managed.ICEBERG).withConfig(configBuilder.build())); + } + + private Map getBaseConfig() { + ImmutableMap.Builder configBuilder = ImmutableMap.builder(); + configBuilder.put("table", tableIdentifier); + if (catalogName != null) { + configBuilder.put(CATALOG_NAME_FIELD, catalogName); + } + if (catalogProps != null) { + configBuilder.put(CATALOG_PROPERTIES_FIELD, catalogProps); + } + if (configProps != null) { + configBuilder.put(HADOOP_CONFIG_PROPERTIES_FIELD, configProps); + } + return configBuilder.build(); + } + + @Override + public PCollection.IsBounded isBounded() { + return PCollection.IsBounded.BOUNDED; + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java new file mode 100644 index 000000000000..270b1d422327 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java @@ -0,0 +1,34 @@ +/* + * 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.beam.sdk.extensions.sql.meta.provider.iceberg; + +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.provider.InMemoryMetaTableProvider; + +public class IcebergTableProvider extends InMemoryMetaTableProvider { + @Override + public String getTableType() { + return "iceberg"; + } + + @Override + public BeamSqlTable buildBeamSqlTable(Table table) { + return new IcebergTable(table); + } +} diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java new file mode 100644 index 000000000000..abcff645b953 --- /dev/null +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -0,0 +1,175 @@ +/* + * 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.beam.sdk.extensions.sql.meta.provider.iceberg; + +import static org.apache.beam.sdk.extensions.sql.utils.DateTimeUtils.parseTimestampWithUTCTimeZone; +import static org.apache.beam.sdk.schemas.Schema.FieldType.BOOLEAN; +import static org.apache.beam.sdk.schemas.Schema.FieldType.DOUBLE; +import static org.apache.beam.sdk.schemas.Schema.FieldType.FLOAT; +import static org.apache.beam.sdk.schemas.Schema.FieldType.INT32; +import static org.apache.beam.sdk.schemas.Schema.FieldType.INT64; +import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertEquals; + +import com.google.api.services.bigquery.model.TableRow; +import java.io.IOException; +import java.util.Arrays; +import java.util.UUID; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; +import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; +import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Integration tests for writing to Iceberg with Beam SQL. */ +@RunWith(JUnit4.class) +public class IcebergReadWriteIT { + private static final Schema SOURCE_SCHEMA = + Schema.builder() + .addNullableField("c_bigint", INT64) + .addNullableField("c_integer", INT32) + .addNullableField("c_float", FLOAT) + .addNullableField("c_double", DOUBLE) + .addNullableField("c_boolean", BOOLEAN) + .addNullableField("c_timestamp", CalciteUtils.TIMESTAMP) + .addNullableField("c_varchar", STRING) + .addNullableField("c_char", STRING) + .addNullableField("c_arr", Schema.FieldType.array(STRING)) + .build(); + + @Rule public transient TestPipeline writePipeline = TestPipeline.create(); + @Rule public transient TestPipeline readPipeline = TestPipeline.create(); + + private static final BigqueryClient BQ_CLIENT = new BigqueryClient("IcebergReadWriteIT"); + static final String DATASET = "iceberg_sql_tests_" + System.nanoTime(); + static String warehouse; + protected static final GcpOptions OPTIONS = + TestPipeline.testingPipelineOptions().as(GcpOptions.class); + + @BeforeClass + public static void createDataset() throws IOException, InterruptedException { + warehouse = + String.format( + "%s%s/%s", + TestPipeline.testingPipelineOptions().getTempLocation(), + IcebergReadWriteIT.class.getSimpleName(), + UUID.randomUUID()); + BQ_CLIENT.createNewDataset(OPTIONS.getProject(), DATASET); + } + + @AfterClass + public static void deleteDataset() { + BQ_CLIENT.deleteDataset(OPTIONS.getProject(), DATASET); + } + + @Test + public void testSqlWriteAndRead() throws IOException, InterruptedException { + BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(new IcebergTableProvider()); + String tableIdentifier = DATASET + ".my_table"; + + // 1) create beam table + String createTableStatement = + "CREATE EXTERNAL TABLE TEST( \n" + + " c_bigint BIGINT, \n" + + " c_integer INTEGER, \n" + + " c_float FLOAT, \n" + + " c_double DOUBLE, \n" + + " c_boolean BOOLEAN, \n" + + " c_timestamp TIMESTAMP, \n" + + " c_varchar VARCHAR, \n " + + " c_char CHAR, \n" + + " c_arr ARRAY \n" + + ") \n" + + "TYPE 'iceberg' \n" + + "LOCATION '" + + tableIdentifier + + "'\n" + + "TBLPROPERTIES '{\"catalog_properties\": {" + + "\"catalog-impl\": \"org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog\"," + + "\"io-impl\": \"org.apache.iceberg.gcp.gcs.GCSFileIO\"," + + "\"warehouse\": \"" + + warehouse + + "\"," + + "\"gcp_project\": \"" + + OPTIONS.getProject() + + "\"," + + "\"gcp_region\": \"us-central1\"" + + "}}'"; + sqlEnv.executeDdl(createTableStatement); + + // 2) write to underlying Iceberg table + String insertStatement = + "INSERT INTO TEST VALUES (" + + "9223372036854775807, " + + "2147483647, " + + "1.0, " + + "1.0, " + + "TRUE, " + + "TIMESTAMP '2018-05-28 20:17:40.123', " + + "'varchar', " + + "'char', " + + "ARRAY['123', '456']" + + ")"; + sqlEnv.parseQuery(insertStatement); + BeamSqlRelUtils.toPCollection(writePipeline, sqlEnv.parseQuery(insertStatement)); + writePipeline.run().waitUntilFinish(); + + // 3) run external query on Iceberg table (hosted on BQ) to verify correct row was written + String query = String.format("SELECT * FROM `%s.%s`", OPTIONS.getProject(), tableIdentifier); + TableRow returnedRow = + BQ_CLIENT.queryUnflattened(query, OPTIONS.getProject(), true, true).get(0); + Row beamRow = BigQueryUtils.toBeamRow(SOURCE_SCHEMA, returnedRow); + Row expectedRow = + Row.withSchema(SOURCE_SCHEMA) + .addValues( + 9223372036854775807L, + 2147483647, + (float) 1.0, + 1.0, + true, + parseTimestampWithUTCTimeZone("2018-05-28 20:17:40.123"), + "varchar", + "char", + Arrays.asList("123", "456")) + .build(); + assertEquals(expectedRow, beamRow); + + // 4) read using Beam SQL and verify + String selectTableStatement = "SELECT * FROM TEST"; + PCollection output = + BeamSqlRelUtils.toPCollection(readPipeline, sqlEnv.parseQuery(selectTableStatement)); + PAssert.that(output).containsInAnyOrder(expectedRow); + PipelineResult.State state = readPipeline.run().waitUntilFinish(); + assertThat(state, equalTo(PipelineResult.State.DONE)); + } +} diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java new file mode 100644 index 000000000000..f40ca4ad90ef --- /dev/null +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java @@ -0,0 +1,95 @@ +/* + * 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.beam.sdk.extensions.sql.meta.provider.iceberg; + +import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.CATALOG_NAME_FIELD; +import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.CATALOG_PROPERTIES_FIELD; +import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.HADOOP_CONFIG_PROPERTIES_FIELD; +import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.TRIGGERING_FREQUENCY_FIELD; +import static org.apache.beam.sdk.schemas.Schema.toSchema; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; + +import java.util.stream.Stream; +import org.apache.beam.sdk.extensions.sql.TableUtils; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.schemas.Schema; +import org.junit.Test; +import org.testcontainers.shaded.com.fasterxml.jackson.databind.ObjectMapper; +import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; + +/** UnitTest for {@link IcebergTableProvider}. */ +public class IcebergTableProviderTest { + private IcebergTableProvider provider = new IcebergTableProvider(); + + @Test + public void testGetTableType() { + assertEquals("iceberg", provider.getTableType()); + } + + @Test + public void testBuildBeamSqlTable() throws Exception { + ImmutableMap properties = + ImmutableMap.of( + CATALOG_PROPERTIES_FIELD, + ImmutableMap.of( + "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", + "warehouse", "gs://bucket/warehouse"), + HADOOP_CONFIG_PROPERTIES_FIELD, + ImmutableMap.of( + "fs.gs.project.id", "apache-beam-testing", + "foo", "bar"), + TRIGGERING_FREQUENCY_FIELD, + 30, + CATALOG_NAME_FIELD, + "my_catalog"); + + ObjectMapper mapper = new ObjectMapper(); + String propertiesString = mapper.writeValueAsString(properties); + Table table = fakeTableWithProperties("my_table", propertiesString); + BeamSqlTable sqlTable = provider.buildBeamSqlTable(table); + + assertNotNull(sqlTable); + assertTrue(sqlTable instanceof IcebergTable); + + IcebergTable icebergTable = (IcebergTable) sqlTable; + assertEquals("namespace.table", icebergTable.tableIdentifier); + assertEquals(properties.get(CATALOG_NAME_FIELD), icebergTable.catalogName); + assertEquals(properties.get(CATALOG_PROPERTIES_FIELD), icebergTable.catalogProps); + assertEquals(properties.get(HADOOP_CONFIG_PROPERTIES_FIELD), icebergTable.configProps); + assertEquals(properties.get(TRIGGERING_FREQUENCY_FIELD), icebergTable.triggeringFrequency); + } + + private static Table fakeTableWithProperties(String name, String properties) { + return Table.builder() + .name(name) + .comment(name + " table") + .location("namespace.table") + .schema( + Stream.of( + Schema.Field.nullable("id", Schema.FieldType.INT32), + Schema.Field.nullable("name", Schema.FieldType.STRING)) + .collect(toSchema())) + .type("iceberg") + .properties(TableUtils.parseProperties(properties)) + .build(); + } +} From f1cbc811e34e84e143443d71ab22dded9d1c6f4f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 1 May 2025 14:02:35 -0400 Subject: [PATCH 02/20] properties go in the tableprovider initialization --- .../meta/provider/iceberg/IcebergTable.java | 41 +++++++------------ .../iceberg/IcebergTableProvider.java | 27 +++++++++++- .../provider/iceberg/IcebergReadWriteIT.java | 25 ++++++----- .../iceberg/IcebergTableProviderTest.java | 38 ++++++++--------- 4 files changed, 72 insertions(+), 59 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java index 8497076637a4..83dfe9959207 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java @@ -19,12 +19,11 @@ import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; -import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable; import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; import org.apache.beam.sdk.managed.Managed; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -43,28 +42,15 @@ class IcebergTable extends SchemaBaseBeamTable { static final String TRIGGERING_FREQUENCY_FIELD = "triggering_frequency_seconds"; @VisibleForTesting final String tableIdentifier; - @VisibleForTesting @Nullable Map catalogProps; - @VisibleForTesting @Nullable Map configProps; - @VisibleForTesting @Nullable String catalogName; + @VisibleForTesting final IcebergCatalogConfig catalogConfig; @VisibleForTesting @Nullable Integer triggeringFrequency; - IcebergTable(Table table) { + IcebergTable(Table table, IcebergCatalogConfig catalogConfig) { super(table.getSchema()); this.schema = table.getSchema(); this.tableIdentifier = checkArgumentNotNull(table.getLocation()); + this.catalogConfig = catalogConfig; ObjectNode properties = table.getProperties(); - ObjectMapper objectMapper = new ObjectMapper(); - if (properties.has(CATALOG_PROPERTIES_FIELD)) { - this.catalogProps = - objectMapper.convertValue(properties.get(CATALOG_PROPERTIES_FIELD), HashMap.class); - } - if (properties.has(HADOOP_CONFIG_PROPERTIES_FIELD)) { - this.configProps = - objectMapper.convertValue(properties.get(HADOOP_CONFIG_PROPERTIES_FIELD), HashMap.class); - } - if (properties.has(CATALOG_NAME_FIELD)) { - this.catalogName = properties.get(CATALOG_NAME_FIELD).asText(); - } if (properties.has(TRIGGERING_FREQUENCY_FIELD)) { this.triggeringFrequency = properties.get(TRIGGERING_FREQUENCY_FIELD).asInt(); } @@ -88,18 +74,21 @@ public POutput buildIOWriter(PCollection input) { } private Map getBaseConfig() { - ImmutableMap.Builder configBuilder = ImmutableMap.builder(); - configBuilder.put("table", tableIdentifier); - if (catalogName != null) { - configBuilder.put(CATALOG_NAME_FIELD, catalogName); + ImmutableMap.Builder managedConfigBuilder = ImmutableMap.builder(); + managedConfigBuilder.put("table", tableIdentifier); + @Nullable String name = catalogConfig.getCatalogName(); + @Nullable Map catalogProps = catalogConfig.getCatalogProperties(); + @Nullable Map hadoopConfProps = catalogConfig.getConfigProperties(); + if (name != null) { + managedConfigBuilder.put(CATALOG_NAME_FIELD, name); } if (catalogProps != null) { - configBuilder.put(CATALOG_PROPERTIES_FIELD, catalogProps); + managedConfigBuilder.put(CATALOG_PROPERTIES_FIELD, catalogProps); } - if (configProps != null) { - configBuilder.put(HADOOP_CONFIG_PROPERTIES_FIELD, configProps); + if (hadoopConfProps != null) { + managedConfigBuilder.put(HADOOP_CONFIG_PROPERTIES_FIELD, hadoopConfProps); } - return configBuilder.build(); + return managedConfigBuilder.build(); } @Override diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java index 270b1d422327..51878a89fb5e 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java @@ -17,11 +17,36 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; +import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.provider.InMemoryMetaTableProvider; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; public class IcebergTableProvider extends InMemoryMetaTableProvider { + private IcebergCatalogConfig catalogConfig = IcebergCatalogConfig.builder().build(); + + private IcebergTableProvider() {} + + public static IcebergTableProvider create() { + return new IcebergTableProvider(); + } + + public IcebergTableProvider withCatalogProperties(Map catalogProperties) { + catalogConfig = catalogConfig.toBuilder().setCatalogProperties(catalogProperties).build(); + return this; + } + + public IcebergTableProvider withHadoopConfProperties(Map hadoopConfProperties) { + catalogConfig = catalogConfig.toBuilder().setConfigProperties(hadoopConfProperties).build(); + return this; + } + + public IcebergTableProvider withCatalogName(String name) { + catalogConfig = catalogConfig.toBuilder().setCatalogName(name).build(); + return this; + } + @Override public String getTableType() { return "iceberg"; @@ -29,6 +54,6 @@ public String getTableType() { @Override public BeamSqlTable buildBeamSqlTable(Table table) { - return new IcebergTable(table); + return new IcebergTable(table, catalogConfig); } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index abcff645b953..b60ffdd638a6 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -50,6 +50,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; /** Integration tests for writing to Iceberg with Beam SQL. */ @RunWith(JUnit4.class) @@ -94,7 +95,16 @@ public static void deleteDataset() { @Test public void testSqlWriteAndRead() throws IOException, InterruptedException { - BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(new IcebergTableProvider()); + BeamSqlEnv sqlEnv = + BeamSqlEnv.inMemory( + IcebergTableProvider.create() + .withCatalogProperties( + ImmutableMap.of( + "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", + "warehouse", warehouse, + "gcp_project", OPTIONS.getProject(), + "gcp_region", "us-central1"))); String tableIdentifier = DATASET + ".my_table"; // 1) create beam table @@ -113,18 +123,7 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { + "TYPE 'iceberg' \n" + "LOCATION '" + tableIdentifier - + "'\n" - + "TBLPROPERTIES '{\"catalog_properties\": {" - + "\"catalog-impl\": \"org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog\"," - + "\"io-impl\": \"org.apache.iceberg.gcp.gcs.GCSFileIO\"," - + "\"warehouse\": \"" - + warehouse - + "\"," - + "\"gcp_project\": \"" - + OPTIONS.getProject() - + "\"," - + "\"gcp_region\": \"us-central1\"" - + "}}'"; + + "'"; sqlEnv.executeDdl(createTableStatement); // 2) write to underlying Iceberg table diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java index f40ca4ad90ef..255419abe5e3 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.extensions.sql.TableUtils; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; import org.apache.beam.sdk.schemas.Schema; import org.junit.Test; import org.testcontainers.shaded.com.fasterxml.jackson.databind.ObjectMapper; @@ -37,7 +38,18 @@ /** UnitTest for {@link IcebergTableProvider}. */ public class IcebergTableProviderTest { - private IcebergTableProvider provider = new IcebergTableProvider(); + private final IcebergTableProvider provider = + IcebergTableProvider.create() + .withCatalogProperties( + ImmutableMap.of( + "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", + "warehouse", "gs://bucket/warehouse")) + .withHadoopConfProperties( + ImmutableMap.of( + "fs.gs.project.id", "apache-beam-testing", + "foo", "bar")) + .withCatalogName("my_catalog"); @Test public void testGetTableType() { @@ -46,21 +58,7 @@ public void testGetTableType() { @Test public void testBuildBeamSqlTable() throws Exception { - ImmutableMap properties = - ImmutableMap.of( - CATALOG_PROPERTIES_FIELD, - ImmutableMap.of( - "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", - "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", - "warehouse", "gs://bucket/warehouse"), - HADOOP_CONFIG_PROPERTIES_FIELD, - ImmutableMap.of( - "fs.gs.project.id", "apache-beam-testing", - "foo", "bar"), - TRIGGERING_FREQUENCY_FIELD, - 30, - CATALOG_NAME_FIELD, - "my_catalog"); + ImmutableMap properties = ImmutableMap.of(TRIGGERING_FREQUENCY_FIELD, 30); ObjectMapper mapper = new ObjectMapper(); String propertiesString = mapper.writeValueAsString(properties); @@ -71,10 +69,12 @@ public void testBuildBeamSqlTable() throws Exception { assertTrue(sqlTable instanceof IcebergTable); IcebergTable icebergTable = (IcebergTable) sqlTable; + IcebergCatalogConfig catalogConfig = icebergTable.catalogConfig; assertEquals("namespace.table", icebergTable.tableIdentifier); - assertEquals(properties.get(CATALOG_NAME_FIELD), icebergTable.catalogName); - assertEquals(properties.get(CATALOG_PROPERTIES_FIELD), icebergTable.catalogProps); - assertEquals(properties.get(HADOOP_CONFIG_PROPERTIES_FIELD), icebergTable.configProps); + assertEquals(properties.get(CATALOG_NAME_FIELD), catalogConfig.getCatalogName()); + assertEquals(properties.get(CATALOG_PROPERTIES_FIELD), catalogConfig.getCatalogProperties()); + assertEquals( + properties.get(HADOOP_CONFIG_PROPERTIES_FIELD), catalogConfig.getConfigProperties()); assertEquals(properties.get(TRIGGERING_FREQUENCY_FIELD), icebergTable.triggeringFrequency); } From 2f4bfecfc802403cb170f30ccd01ca1e6b8f385f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 1 May 2025 14:05:36 -0400 Subject: [PATCH 03/20] tobuilder --- .../org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java index c3a185f3a833..3358bc072a2d 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCatalogConfig.java @@ -50,6 +50,8 @@ public static Builder builder() { return new AutoValue_IcebergCatalogConfig.Builder(); } + public abstract Builder toBuilder(); + public org.apache.iceberg.catalog.Catalog catalog() { if (cachedCatalog == null) { String catalogName = getCatalogName(); From b7e85c1e667ffbdf8d5243394baad48ce138cc84 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 6 May 2025 16:46:56 -0400 Subject: [PATCH 04/20] streaming integration test --- sdks/java/extensions/sql/build.gradle | 2 +- .../beam/sdk/extensions/sql/SqlTransform.java | 6 + .../iceberg/IcebergTableProvider.java | 2 - .../sql/meta/store/InMemoryMetaStore.java | 4 + .../sdk/extensions/sql/PubsubToIcebergIT.java | 249 ++++++++++++++++++ .../provider/iceberg/IcebergReadWriteIT.java | 2 +- 6 files changed, 261 insertions(+), 4 deletions(-) create mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index 72cae1c2031c..2b6197401e1b 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -76,7 +76,7 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(":sdks:java:managed") implementation project(":sdks:java:io:iceberg") - implementation project(":sdks:java:io:iceberg:bqms") + runtimeOnly project(":sdks:java:io:iceberg:bqms") implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:join-library") permitUnusedDeclared project(":sdks:java:extensions:join-library") // BEAM-11761 diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java index 07103f2ab714..98c453da49d9 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java @@ -149,6 +149,12 @@ public PCollection expand(PInput input) { sqlEnvBuilder.autoLoadUserDefinedFunctions(); ServiceLoader.load(TableProvider.class).forEach(metaTableProvider::registerProvider); } + // register user-specified providers + for (Map.Entry entry : tableProviderMap().entrySet()) { + if (!metaTableProvider.hasProviderForType(entry.getKey())) { + metaTableProvider.registerProvider(entry.getValue()); + } + } tableProviderMap().forEach(sqlEnvBuilder::addSchema); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java index 51878a89fb5e..0fd1c7f45aa2 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java @@ -26,8 +26,6 @@ public class IcebergTableProvider extends InMemoryMetaTableProvider { private IcebergCatalogConfig catalogConfig = IcebergCatalogConfig.builder().build(); - private IcebergTableProvider() {} - public static IcebergTableProvider create() { return new IcebergTableProvider(); } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java index 8a892cc2eb73..d8f753840989 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java @@ -113,4 +113,8 @@ private void initTablesFromProvider(TableProvider provider) { Map getProviders() { return providers; } + + public boolean hasProviderForType(String type) { + return getProviders().containsKey(type); + } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java new file mode 100644 index 000000000000..346baf7007fb --- /dev/null +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java @@ -0,0 +1,249 @@ +/* + * 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.beam.sdk.extensions.sql; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.beam.sdk.schemas.Schema.FieldType.INT64; +import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING; + +import com.google.api.services.bigquery.model.TableRow; +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergReadWriteIT; +import org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTableProvider; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; +import org.apache.beam.sdk.io.gcp.pubsub.TestPubsub; +import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.util.BackOff; +import org.apache.beam.sdk.util.BackOffUtils; +import org.apache.beam.sdk.util.FluentBackoff; +import org.apache.beam.sdk.util.Sleeper; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TestName; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests stream writing to Iceberg with Beam SQL. */ +@RunWith(JUnit4.class) +public class PubsubToIcebergIT implements Serializable { + private static final Schema SOURCE_SCHEMA = + Schema.builder().addNullableField("id", INT64).addNullableField("name", STRING).build(); + + @Rule public transient TestPipeline pipeline = TestPipeline.create(); + @Rule public transient TestPubsub pubsub = TestPubsub.create(); + + private static final BigqueryClient BQ_CLIENT = new BigqueryClient("PubsubToIcebergIT"); + static final String DATASET = "sql_pubsub_to_iceberg_it_" + System.nanoTime(); + static String warehouse; + protected static final GcpOptions OPTIONS = + TestPipeline.testingPipelineOptions().as(GcpOptions.class); + @Rule public TestName testName = new TestName(); + + @BeforeClass + public static void createDataset() throws IOException, InterruptedException { + warehouse = + String.format( + "%s%s/%s", + TestPipeline.testingPipelineOptions().getTempLocation(), + IcebergReadWriteIT.class.getSimpleName(), + UUID.randomUUID()); + BQ_CLIENT.createNewDataset(OPTIONS.getProject(), DATASET); + } + + private String tableIdentifier; + private Map icebergConfig; + + @Before + public void setup() { + tableIdentifier = DATASET + "." + testName.getMethodName(); + icebergConfig = + ImmutableMap.of( + "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", + "warehouse", warehouse, + "gcp_project", OPTIONS.getProject(), + "gcp_region", "us-central1"); + } + + @AfterClass + public static void deleteDataset() { + BQ_CLIENT.deleteDataset(OPTIONS.getProject(), DATASET); + } + + @Test + public void testSimpleInsert() throws Exception { + String pubsubTableString = + "CREATE EXTERNAL TABLE pubsub_topic (\n" + + "event_timestamp TIMESTAMP, \n" + + "attributes MAP, \n" + + "payload ROW< \n" + + " id BIGINT, \n" + + " name VARCHAR \n" + + " > \n" + + ") \n" + + "TYPE 'pubsub' \n" + + "LOCATION '" + + pubsub.topicPath() + + "' \n" + + "TBLPROPERTIES '{ \"timestampAttributeKey\" : \"ts\" }'"; + String icebergTableString = + "CREATE EXTERNAL TABLE iceberg_table( \n" + + " id BIGINT, \n" + + " name VARCHAR \n " + + ") \n" + + "TYPE 'iceberg' \n" + + "LOCATION '" + + tableIdentifier + + "' \n" + + "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'"; + String insertStatement = + "INSERT INTO iceberg_table \n" + + "SELECT \n" + + " pubsub_topic.payload.id, \n" + + " pubsub_topic.payload.name \n" + + "FROM pubsub_topic"; + pipeline.apply( + SqlTransform.query(insertStatement) + .withDdlString(pubsubTableString) + .withDdlString(icebergTableString) + .withTableProvider( + "iceberg", IcebergTableProvider.create().withCatalogProperties(icebergConfig))); + pipeline.run(); + + // Block until a subscription for this topic exists + pubsub.assertSubscriptionEventuallyCreated( + pipeline.getOptions().as(GcpOptions.class).getProject(), Duration.standardMinutes(5)); + + List messages = + ImmutableList.of( + message(ts(1), 3, "foo"), message(ts(2), 5, "bar"), message(ts(3), 7, "baz")); + pubsub.publish(messages); + + validateRowsWritten(); + } + + @Test + public void testSimpleInsertFlat() throws Exception { + String pubsubTableString = + "CREATE EXTERNAL TABLE pubsub_topic (\n" + + "event_timestamp TIMESTAMP, \n" + + "id BIGINT, \n" + + "name VARCHAR \n" + + ") \n" + + "TYPE 'pubsub' \n" + + "LOCATION '" + + pubsub.topicPath() + + "' \n" + + "TBLPROPERTIES '{ \"timestampAttributeKey\" : \"ts\" }'"; + String bqTableString = + "CREATE EXTERNAL TABLE iceberg_table( \n" + + " id BIGINT, \n" + + " name VARCHAR \n " + + ") \n" + + "TYPE 'iceberg' \n" + + "LOCATION '" + + tableIdentifier + + "' \n" + + "TBLPROPERTIES '{ \"triggering_frequency_seconds\" : 10 }'"; + String insertStatement = + "INSERT INTO iceberg_table \n" + + "SELECT \n" + + " id, \n" + + " name \n" + + "FROM pubsub_topic"; + + pipeline.apply( + SqlTransform.query(insertStatement) + .withDdlString(pubsubTableString) + .withDdlString(bqTableString) + .withTableProvider( + "iceberg", IcebergTableProvider.create().withCatalogProperties(icebergConfig))); + pipeline.run(); + + validateRowsWritten(); + } + + private void validateRowsWritten() throws IOException, InterruptedException { + String query = String.format("SELECT * FROM `%s.%s`", OPTIONS.getProject(), tableIdentifier); + List expectedRows = + ImmutableList.of( + row(SOURCE_SCHEMA, 3L, "foo"), + row(SOURCE_SCHEMA, 5L, "bar"), + row(SOURCE_SCHEMA, 7L, "baz")); + + BackOff backOff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.standardSeconds(10)) + .withMaxBackoff(Duration.standardSeconds(20)) + .withMaxCumulativeBackoff(Duration.standardMinutes(5)) + .backoff(); + Sleeper sleeper = Sleeper.DEFAULT; + do { + List returnedRows = ImmutableList.of(); + try { + returnedRows = BQ_CLIENT.queryUnflattened(query, OPTIONS.getProject(), true, true); + } catch (Exception ignored) { + } + List beamRows = + returnedRows.stream() + .map(r -> BigQueryUtils.toBeamRow(SOURCE_SCHEMA, r)) + .collect(Collectors.toList()); + if (beamRows.containsAll(expectedRows)) { + break; + } + } while (BackOffUtils.next(sleeper, backOff)); + } + + private Row row(Schema schema, Object... values) { + return Row.withSchema(schema).addValues(values).build(); + } + + private PubsubMessage message(Instant timestamp, int id, String name) { + return message(timestamp, jsonString(id, name)); + } + + private PubsubMessage message(Instant timestamp, String jsonPayload) { + return new PubsubMessage( + jsonPayload.getBytes(UTF_8), ImmutableMap.of("ts", String.valueOf(timestamp.getMillis()))); + } + + private String jsonString(int id, String name) { + return "{ \"id\" : " + id + ", \"name\" : \"" + name + "\" }"; + } + + private Instant ts(long millis) { + return new Instant(millis); + } +} diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index b60ffdd638a6..e5148ff07548 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -44,13 +44,13 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; /** Integration tests for writing to Iceberg with Beam SQL. */ @RunWith(JUnit4.class) From 19d3db12066aad5d0e90395b7fb4514f76e58d74 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 6 May 2025 16:57:27 -0400 Subject: [PATCH 05/20] spotless --- .../sql/meta/provider/iceberg/IcebergTableProviderTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java index 255419abe5e3..c5aa3d130a72 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java @@ -32,9 +32,9 @@ import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.vendor.calcite.v1_28_0.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Test; -import org.testcontainers.shaded.com.fasterxml.jackson.databind.ObjectMapper; -import org.testcontainers.shaded.com.google.common.collect.ImmutableMap; /** UnitTest for {@link IcebergTableProvider}. */ public class IcebergTableProviderTest { From 87b838eec4c250702604b242fa591a51e68c662c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 8 May 2025 12:08:19 -0400 Subject: [PATCH 06/20] extend test to include multi nested types; fix iceberg <-> conversion logic --- .../meta/provider/iceberg/package-info.java | 20 +++ .../sdk/extensions/sql/PubsubToIcebergIT.java | 3 +- .../provider/iceberg/IcebergReadWriteIT.java | 33 ++++- .../beam/sdk/io/iceberg/IcebergUtils.java | 73 ++++++++++- .../sdk/io/iceberg/RecordWriterManager.java | 2 + .../beam/sdk/io/iceberg/IcebergUtilsTest.java | 120 ++++++++++++++++++ 6 files changed, 241 insertions(+), 10 deletions(-) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/package-info.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/package-info.java new file mode 100644 index 000000000000..01a34920f585 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Table schema for Iceberg. */ +package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java index 346baf7007fb..1fd619a684c4 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java @@ -29,7 +29,6 @@ import java.util.UUID; import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; -import org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergReadWriteIT; import org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTableProvider; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; @@ -77,7 +76,7 @@ public static void createDataset() throws IOException, InterruptedException { String.format( "%s%s/%s", TestPipeline.testingPipelineOptions().getTempLocation(), - IcebergReadWriteIT.class.getSimpleName(), + PubsubToIcebergIT.class.getSimpleName(), UUID.randomUUID()); BQ_CLIENT.createNewDataset(OPTIONS.getProject(), DATASET); } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index e5148ff07548..0cfe64e8c13b 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; +import static java.util.Arrays.asList; import static org.apache.beam.sdk.extensions.sql.utils.DateTimeUtils.parseTimestampWithUTCTimeZone; import static org.apache.beam.sdk.schemas.Schema.FieldType.BOOLEAN; import static org.apache.beam.sdk.schemas.Schema.FieldType.DOUBLE; @@ -24,13 +25,15 @@ import static org.apache.beam.sdk.schemas.Schema.FieldType.INT32; import static org.apache.beam.sdk.schemas.Schema.FieldType.INT64; import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING; +import static org.apache.beam.sdk.schemas.Schema.FieldType.array; +import static org.apache.beam.sdk.schemas.Schema.FieldType.row; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; import com.google.api.services.bigquery.model.TableRow; import java.io.IOException; -import java.util.Arrays; +import java.util.List; import java.util.UUID; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; @@ -55,6 +58,11 @@ /** Integration tests for writing to Iceberg with Beam SQL. */ @RunWith(JUnit4.class) public class IcebergReadWriteIT { + private static final Schema NESTED_SCHEMA = + Schema.builder() + .addNullableArrayField("c_arr_struct_arr", STRING) + .addNullableInt32Field("c_arr_struct_integer") + .build(); private static final Schema SOURCE_SCHEMA = Schema.builder() .addNullableField("c_bigint", INT64) @@ -65,7 +73,8 @@ public class IcebergReadWriteIT { .addNullableField("c_timestamp", CalciteUtils.TIMESTAMP) .addNullableField("c_varchar", STRING) .addNullableField("c_char", STRING) - .addNullableField("c_arr", Schema.FieldType.array(STRING)) + .addNullableField("c_arr", array(STRING)) + .addNullableField("c_arr_struct", array(row(NESTED_SCHEMA))) .build(); @Rule public transient TestPipeline writePipeline = TestPipeline.create(); @@ -118,7 +127,8 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { + " c_timestamp TIMESTAMP, \n" + " c_varchar VARCHAR, \n " + " c_char CHAR, \n" - + " c_arr ARRAY \n" + + " c_arr ARRAY, \n" + + " c_arr_struct ARRAY, c_arr_struct_integer INTEGER>> \n" + ") \n" + "TYPE 'iceberg' \n" + "LOCATION '" @@ -137,7 +147,11 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { + "TIMESTAMP '2018-05-28 20:17:40.123', " + "'varchar', " + "'char', " - + "ARRAY['123', '456']" + + "ARRAY['123', '456'], " + + "ARRAY[" + + "ROW(ARRAY['abc', 'xyz'], 123), " + + "ROW(ARRAY['foo', 'bar'], 456), " + + "ROW(ARRAY['cat', 'dog'], 789)]" + ")"; sqlEnv.parseQuery(insertStatement); BeamSqlRelUtils.toPCollection(writePipeline, sqlEnv.parseQuery(insertStatement)); @@ -159,7 +173,11 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { parseTimestampWithUTCTimeZone("2018-05-28 20:17:40.123"), "varchar", "char", - Arrays.asList("123", "456")) + asList("123", "456"), + asList( + nestedRow(asList("abc", "xyz"), 123), + nestedRow(asList("foo", "bar"), 456), + nestedRow(asList("cat", "dog"), 789))) .build(); assertEquals(expectedRow, beamRow); @@ -171,4 +189,9 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { PipelineResult.State state = readPipeline.run().waitUntilFinish(); assertThat(state, equalTo(PipelineResult.State.DONE)); } + + private Row nestedRow(List arr, Integer intVal) { + System.out.println(arr); + return Row.withSchema(NESTED_SCHEMA).addValues(arr, intVal).build(); + } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java index ef19a5881366..9f9df1f08beb 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.io.iceberg; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import java.nio.ByteBuffer; import java.time.LocalDate; @@ -26,10 +28,12 @@ import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; import org.apache.beam.sdk.util.Preconditions; @@ -41,6 +45,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; +import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.joda.time.Instant; @@ -346,10 +351,33 @@ private static void copyFieldIntoRecord(Record rec, Types.NestedField field, Row copyRowIntoRecord(GenericRecord.create(field.type().asStructType()), row))); break; case LIST: - Optional.ofNullable(value.getArray(name)).ifPresent(list -> rec.setField(name, list)); + Collection<@NonNull ?> icebergList = value.getArray(name); + Type collectionType = ((Types.ListType) field.type()).elementType(); + + if (collectionType.isStructType() && icebergList != null) { + org.apache.iceberg.Schema innerSchema = collectionType.asStructType().asSchema(); + icebergList = + icebergList.stream() + .map(v -> beamRowToIcebergRecord(innerSchema, (Row) v)) + .collect(Collectors.toList()); + } + Optional.ofNullable(icebergList).ifPresent(list -> rec.setField(name, list)); break; case MAP: - Optional.ofNullable(value.getMap(name)).ifPresent(v -> rec.setField(name, v)); + Map icebergMap = value.getMap(name); + Type valueType = ((Types.MapType) field.type()).valueType(); + // recurse on struct types + if (valueType.isStructType() && icebergMap != null) { + org.apache.iceberg.Schema innerSchema = valueType.asStructType().asSchema(); + + ImmutableMap.Builder newMap = ImmutableMap.builder(); + for (Map.Entry entry : icebergMap.entrySet()) { + Row row = checkStateNotNull(((Row) entry.getValue())); + newMap.put(checkStateNotNull(entry.getKey()), beamRowToIcebergRecord(innerSchema, row)); + } + icebergMap = newMap.build(); + } + Optional.ofNullable(icebergMap).ifPresent(v -> rec.setField(name, v)); break; } } @@ -427,10 +455,49 @@ public static Row icebergRecordToBeamRow(Schema schema, Record record) { case DOUBLE: // Iceberg and Beam both use double case STRING: // Iceberg and Beam both use String case BOOLEAN: // Iceberg and Beam both use boolean + rowBuilder.addValue(icebergValue); + break; case ARRAY: case ITERABLE: + checkState( + icebergValue instanceof List, + "Expected List type for field '%s' but received %s", + field.getName(), + icebergValue.getClass()); + List<@NonNull ?> beamList = (List<@NonNull ?>) icebergValue; + Schema.FieldType collectionType = + checkStateNotNull(field.getType().getCollectionElementType()); + // recurse on struct types + if (collectionType.getTypeName().isCompositeType()) { + Schema innerSchema = checkStateNotNull(collectionType.getRowSchema()); + beamList = + beamList.stream() + .map(v -> icebergRecordToBeamRow(innerSchema, (Record) v)) + .collect(Collectors.toList()); + } + rowBuilder.addValue(beamList); + break; case MAP: - rowBuilder.addValue(icebergValue); + checkState( + icebergValue instanceof Map, + "Expected Map type for field '%s' but received %s", + field.getName(), + icebergValue.getClass()); + Map beamMap = (Map) icebergValue; + Schema.FieldType valueType = checkStateNotNull(field.getType().getMapValueType()); + // recurse on struct types + if (valueType.getTypeName().isCompositeType()) { + Schema innerSchema = checkStateNotNull(valueType.getRowSchema()); + ImmutableMap.Builder newMap = ImmutableMap.builder(); + for (Map.Entry entry : ((Map) icebergValue).entrySet()) { + Record rec = ((Record) entry.getValue()); + newMap.put( + checkStateNotNull(entry.getKey()), + icebergRecordToBeamRow(innerSchema, checkStateNotNull(rec))); + } + beamMap = newMap.build(); + } + rowBuilder.addValue(beamMap); break; case DATETIME: // Iceberg uses a long for micros. diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java index bbde6c9e864e..97539c48a7fa 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java @@ -324,6 +324,8 @@ public boolean write(WindowedValue icebergDestination, Row r }); Record icebergRecord = IcebergUtils.beamRowToIcebergRecord(destinationState.schema, row); + System.out.println("beam row: " + row); + System.out.println("ice record: " + icebergRecord); return destinationState.write(icebergRecord); } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java index 134f05c34bfb..3d2928c8c659 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java @@ -242,6 +242,20 @@ public void testList() { Types.ListType.ofRequired(1, Types.StringType.get()), list); } + + @Test + public void testListOfRecords() { + Record actual = + IcebergUtils.beamRowToIcebergRecord(RECORD_LIST_ICEBERG_SCHEMA, ROW_LIST_OF_ROWS); + assertEquals(RECORD_LIST_OF_RECORDS, actual); + } + + @Test + public void testMapOfRecords() { + Record actual = + IcebergUtils.beamRowToIcebergRecord(RECORD_MAP_ICEBERG_SCHEMA, ROW_MAP_OF_ROWS); + assertEquals(RECORD_MAP_OF_RECORDS, actual); + } } @RunWith(JUnit4.class) @@ -418,8 +432,114 @@ public void testList() { Schema.FieldType.iterable(Schema.FieldType.STRING), list); } + + @Test + public void testListOfRecords() { + Row actual = + IcebergUtils.icebergRecordToBeamRow(ROW_LIST_BEAM_SCHEMA, RECORD_LIST_OF_RECORDS); + assertEquals(ROW_LIST_OF_ROWS, actual); + } + + @Test + public void testMapOfRecords() { + Row actual = IcebergUtils.icebergRecordToBeamRow(ROW_MAP_BEAM_SCHEMA, RECORD_MAP_OF_RECORDS); + assertEquals(ROW_MAP_OF_ROWS, actual); + } } + static Schema NESTED_BEAM_SCHEMA = + Schema.builder() + .addArrayField("str_list", Schema.FieldType.STRING) + .addInt32Field("int") + .build(); + static Schema ROW_LIST_BEAM_SCHEMA = + Schema.builder() + .addArrayField("list", Schema.FieldType.row(NESTED_BEAM_SCHEMA)) + .addBooleanField("bool") + .build(); + static Schema ROW_MAP_BEAM_SCHEMA = + Schema.builder() + .addMapField("map", Schema.FieldType.STRING, Schema.FieldType.row(NESTED_BEAM_SCHEMA)) + .build(); + static org.apache.iceberg.Schema NESTED_ICEBERG_SCHEMA = + new org.apache.iceberg.Schema( + required(4, "str_list", Types.ListType.ofRequired(6, Types.StringType.get())), + required(5, "int", Types.IntegerType.get())); + static org.apache.iceberg.Schema RECORD_LIST_ICEBERG_SCHEMA = + new org.apache.iceberg.Schema( + required( + 1, + "list", + Types.ListType.ofRequired(3, Types.StructType.of(NESTED_ICEBERG_SCHEMA.columns()))), + required(2, "bool", Types.BooleanType.get())); + static org.apache.iceberg.Schema RECORD_MAP_ICEBERG_SCHEMA = + new org.apache.iceberg.Schema( + required( + 1, + "map", + Types.MapType.ofRequired( + 2, + 3, + Types.StringType.get(), + Types.StructType.of(NESTED_ICEBERG_SCHEMA.columns())))); + static Record RECORD_LIST_OF_RECORDS = + GenericRecord.create(RECORD_LIST_ICEBERG_SCHEMA) + .copy( + ImmutableMap.of( + "list", + Arrays.asList( + GenericRecord.create(NESTED_ICEBERG_SCHEMA) + .copy( + ImmutableMap.of( + "str_list", Arrays.asList("a", "b", "c"), "int", 123)), + GenericRecord.create(NESTED_ICEBERG_SCHEMA) + .copy( + ImmutableMap.of( + "str_list", Arrays.asList("x", "y", "z"), "int", 789))), + "bool", + true)); + static Row ROW_LIST_OF_ROWS = + Row.withSchema(ROW_LIST_BEAM_SCHEMA) + .addValues( + Arrays.asList( + Row.withSchema(NESTED_BEAM_SCHEMA) + .addValues(Arrays.asList("a", "b", "c"), 123) + .build(), + Row.withSchema(NESTED_BEAM_SCHEMA) + .addValues(Arrays.asList("x", "y", "z"), 789) + .build()), + true) + .build(); + static Record RECORD_MAP_OF_RECORDS = + GenericRecord.create(RECORD_MAP_ICEBERG_SCHEMA) + .copy( + ImmutableMap.of( + "map", + ImmutableMap.of( + "key_1", + GenericRecord.create(NESTED_ICEBERG_SCHEMA) + .copy( + ImmutableMap.of( + "str_list", Arrays.asList("a", "b", "c"), "int", 123)), + "key_2", + GenericRecord.create(NESTED_ICEBERG_SCHEMA) + .copy( + ImmutableMap.of( + "str_list", Arrays.asList("x", "y", "z"), "int", 789))))); + static Row ROW_MAP_OF_ROWS = + Row.withSchema(ROW_MAP_BEAM_SCHEMA) + .addValues( + ImmutableMap.of( + "key_1", + Row.withSchema(NESTED_BEAM_SCHEMA) + .addValues(Arrays.asList("a", "b", "c"), 123) + .build(), + "key_2", + Row.withSchema(NESTED_BEAM_SCHEMA) + .addValues(Arrays.asList("x", "y", "z"), 789) + .build())) + .build(); + @RunWith(JUnit4.class) public static class SchemaTests { private static class BeamFieldTypeTestCase { From 4e4e31f54de69bb6c63443334d10f55f44362981 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 8 May 2025 12:13:09 -0400 Subject: [PATCH 07/20] add to changes.md --- CHANGES.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CHANGES.md b/CHANGES.md index bb0dae00b46c..c709f2934d16 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -72,6 +72,8 @@ ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* [IcebergIO] Now available with Beam SQL! ([#34799](https://github.com/apache/beam/pull/34799)) +* [IcebergIO] Added support for column pruning ([#34856](https://github.com/apache/beam/pull/34856)) ## New Features / Improvements From 06be2c0440de1f4a50cb65355811705f6802dbbd Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 8 May 2025 12:31:23 -0400 Subject: [PATCH 08/20] spotless --- .../apache/beam/sdk/io/iceberg/IcebergUtilsTest.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java index 3d2928c8c659..19f7de1d8897 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java @@ -447,32 +447,32 @@ public void testMapOfRecords() { } } - static Schema NESTED_BEAM_SCHEMA = + static final Schema NESTED_BEAM_SCHEMA = Schema.builder() .addArrayField("str_list", Schema.FieldType.STRING) .addInt32Field("int") .build(); - static Schema ROW_LIST_BEAM_SCHEMA = + static final Schema ROW_LIST_BEAM_SCHEMA = Schema.builder() .addArrayField("list", Schema.FieldType.row(NESTED_BEAM_SCHEMA)) .addBooleanField("bool") .build(); - static Schema ROW_MAP_BEAM_SCHEMA = + static final Schema ROW_MAP_BEAM_SCHEMA = Schema.builder() .addMapField("map", Schema.FieldType.STRING, Schema.FieldType.row(NESTED_BEAM_SCHEMA)) .build(); - static org.apache.iceberg.Schema NESTED_ICEBERG_SCHEMA = + static final org.apache.iceberg.Schema NESTED_ICEBERG_SCHEMA = new org.apache.iceberg.Schema( required(4, "str_list", Types.ListType.ofRequired(6, Types.StringType.get())), required(5, "int", Types.IntegerType.get())); - static org.apache.iceberg.Schema RECORD_LIST_ICEBERG_SCHEMA = + static final org.apache.iceberg.Schema RECORD_LIST_ICEBERG_SCHEMA = new org.apache.iceberg.Schema( required( 1, "list", Types.ListType.ofRequired(3, Types.StructType.of(NESTED_ICEBERG_SCHEMA.columns()))), required(2, "bool", Types.BooleanType.get())); - static org.apache.iceberg.Schema RECORD_MAP_ICEBERG_SCHEMA = + static final org.apache.iceberg.Schema RECORD_MAP_ICEBERG_SCHEMA = new org.apache.iceberg.Schema( required( 1, From 138b54a0b52bd21ff66075127994301bafa74bc6 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 8 May 2025 12:53:32 -0400 Subject: [PATCH 09/20] fix tests --- .../apache/beam/sdk/extensions/sql/SqlTransform.java | 2 +- .../meta/provider/iceberg/IcebergTableProvider.java | 3 ++- .../extensions/sql/meta/store/InMemoryMetaStore.java | 4 ++-- .../provider/iceberg/IcebergTableProviderTest.java | 11 +---------- .../apache/beam/sdk/io/iceberg/IcebergUtilsTest.java | 8 ++++---- 5 files changed, 10 insertions(+), 18 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java index 98c453da49d9..05f2ccbee0dc 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java @@ -151,7 +151,7 @@ public PCollection expand(PInput input) { } // register user-specified providers for (Map.Entry entry : tableProviderMap().entrySet()) { - if (!metaTableProvider.hasProviderForType(entry.getKey())) { + if (!metaTableProvider.hasProvider(entry.getValue())) { metaTableProvider.registerProvider(entry.getValue()); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java index 0fd1c7f45aa2..575d55044ee8 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java @@ -22,9 +22,10 @@ import org.apache.beam.sdk.extensions.sql.meta.Table; import org.apache.beam.sdk.extensions.sql.meta.provider.InMemoryMetaTableProvider; import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; public class IcebergTableProvider extends InMemoryMetaTableProvider { - private IcebergCatalogConfig catalogConfig = IcebergCatalogConfig.builder().build(); + @VisibleForTesting IcebergCatalogConfig catalogConfig = IcebergCatalogConfig.builder().build(); public static IcebergTableProvider create() { return new IcebergTableProvider(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java index d8f753840989..a571a5d2dbb7 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java @@ -114,7 +114,7 @@ Map getProviders() { return providers; } - public boolean hasProviderForType(String type) { - return getProviders().containsKey(type); + public boolean hasProvider(TableProvider provider) { + return providers.containsKey(provider.getTableType()); } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java index c5aa3d130a72..550d4264a4bc 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java @@ -17,9 +17,6 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; -import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.CATALOG_NAME_FIELD; -import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.CATALOG_PROPERTIES_FIELD; -import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.HADOOP_CONFIG_PROPERTIES_FIELD; import static org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTable.TRIGGERING_FREQUENCY_FIELD; import static org.apache.beam.sdk.schemas.Schema.toSchema; import static org.junit.Assert.assertEquals; @@ -30,7 +27,6 @@ import org.apache.beam.sdk.extensions.sql.TableUtils; import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable; import org.apache.beam.sdk.extensions.sql.meta.Table; -import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.vendor.calcite.v1_28_0.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -69,13 +65,8 @@ public void testBuildBeamSqlTable() throws Exception { assertTrue(sqlTable instanceof IcebergTable); IcebergTable icebergTable = (IcebergTable) sqlTable; - IcebergCatalogConfig catalogConfig = icebergTable.catalogConfig; assertEquals("namespace.table", icebergTable.tableIdentifier); - assertEquals(properties.get(CATALOG_NAME_FIELD), catalogConfig.getCatalogName()); - assertEquals(properties.get(CATALOG_PROPERTIES_FIELD), catalogConfig.getCatalogProperties()); - assertEquals( - properties.get(HADOOP_CONFIG_PROPERTIES_FIELD), catalogConfig.getConfigProperties()); - assertEquals(properties.get(TRIGGERING_FREQUENCY_FIELD), icebergTable.triggeringFrequency); + assertEquals(provider.catalogConfig, icebergTable.catalogConfig); } private static Table fakeTableWithProperties(String name, String properties) { diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java index 19f7de1d8897..927fc93f9e60 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergUtilsTest.java @@ -482,7 +482,7 @@ public void testMapOfRecords() { 3, Types.StringType.get(), Types.StructType.of(NESTED_ICEBERG_SCHEMA.columns())))); - static Record RECORD_LIST_OF_RECORDS = + static final Record RECORD_LIST_OF_RECORDS = GenericRecord.create(RECORD_LIST_ICEBERG_SCHEMA) .copy( ImmutableMap.of( @@ -498,7 +498,7 @@ public void testMapOfRecords() { "str_list", Arrays.asList("x", "y", "z"), "int", 789))), "bool", true)); - static Row ROW_LIST_OF_ROWS = + static final Row ROW_LIST_OF_ROWS = Row.withSchema(ROW_LIST_BEAM_SCHEMA) .addValues( Arrays.asList( @@ -510,7 +510,7 @@ public void testMapOfRecords() { .build()), true) .build(); - static Record RECORD_MAP_OF_RECORDS = + static final Record RECORD_MAP_OF_RECORDS = GenericRecord.create(RECORD_MAP_ICEBERG_SCHEMA) .copy( ImmutableMap.of( @@ -526,7 +526,7 @@ public void testMapOfRecords() { .copy( ImmutableMap.of( "str_list", Arrays.asList("x", "y", "z"), "int", 789))))); - static Row ROW_MAP_OF_ROWS = + static final Row ROW_MAP_OF_ROWS = Row.withSchema(ROW_MAP_BEAM_SCHEMA) .addValues( ImmutableMap.of( From 29599c461209a36f78a8ba93664159f16b0d2f04 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 12 May 2025 18:45:11 -0400 Subject: [PATCH 10/20] clean --- .../org/apache/beam/sdk/io/iceberg/RecordWriterManager.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java index 97539c48a7fa..bbde6c9e864e 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java @@ -324,8 +324,6 @@ public boolean write(WindowedValue icebergDestination, Row r }); Record icebergRecord = IcebergUtils.beamRowToIcebergRecord(destinationState.schema, row); - System.out.println("beam row: " + row); - System.out.println("ice record: " + icebergRecord); return destinationState.write(icebergRecord); } From 964b3387bd315e3fc632b9bfd43412a464166db6 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 13 May 2025 20:08:48 -0400 Subject: [PATCH 11/20] update CHANGES --- CHANGES.md | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.md b/CHANGES.md index c709f2934d16..357cb59a8dc4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -74,6 +74,7 @@ * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * [IcebergIO] Now available with Beam SQL! ([#34799](https://github.com/apache/beam/pull/34799)) * [IcebergIO] Added support for column pruning ([#34856](https://github.com/apache/beam/pull/34856)) +* [IcebergIO] Added support for pushdown filtering ([#34827](https://github.com/apache/beam/pull/34827)) ## New Features / Improvements From 9b5583c0ad3e8df5ba994fbc826c7a61ff734b63 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 28 May 2025 18:35:06 -0400 Subject: [PATCH 12/20] add projection pushdown and column pruning --- .../meta/provider/iceberg/IcebergFilter.java | 144 ++++++++++++++++++ .../meta/provider/iceberg/IcebergTable.java | 96 ++++++++++-- .../provider/iceberg/IcebergFilterTest.java | 122 +++++++++++++++ .../provider/iceberg/IcebergReadWriteIT.java | 85 +++++++++-- .../beam/sdk/io/iceberg/FilterUtils.java | 78 +++++++++- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 2 +- .../sdk/io/iceberg/IcebergScanConfig.java | 66 +++++++- .../beam/sdk/io/iceberg/ReadFromTasks.java | 5 +- .../beam/sdk/io/iceberg/ScanTaskReader.java | 27 ++-- .../beam/sdk/io/iceberg/ScanTaskSource.java | 6 - .../beam/sdk/io/iceberg/FilterUtilsTest.java | 19 +++ .../sdk/io/iceberg/IcebergIOReadTest.java | 14 +- 12 files changed, 596 insertions(+), 68 deletions(-) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java create mode 100644 sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java new file mode 100644 index 000000000000..b3854ced46c6 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilter.java @@ -0,0 +1,144 @@ +/* + * 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.beam.sdk.extensions.sql.meta.provider.iceberg; + +import static org.apache.beam.sdk.io.iceberg.FilterUtils.SUPPORTED_OPS; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.AND; +import static org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind.OR; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.type.SqlTypeName; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.commons.lang3.tuple.Pair; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.checkerframework.checker.nullness.qual.Nullable; + +public class IcebergFilter implements BeamSqlTableFilter { + private @Nullable List supported; + private @Nullable List unsupported; + private final List predicateCNF; + + public IcebergFilter(List predicateCNF) { + this.predicateCNF = predicateCNF; + } + + private void maybeInitialize() { + if (supported != null && unsupported != null) { + return; + } + ImmutableList.Builder supportedBuilder = ImmutableList.builder(); + ImmutableList.Builder unsupportedBuilder = ImmutableList.builder(); + for (RexNode node : predicateCNF) { + if (!node.getType().getSqlTypeName().equals(SqlTypeName.BOOLEAN)) { + throw new IllegalArgumentException( + "Predicate node '" + + node.getClass().getSimpleName() + + "' should be a boolean expression, but was: " + + node.getType().getSqlTypeName()); + } + + if (isSupported(node).getLeft()) { + supportedBuilder.add(node); + } else { + unsupportedBuilder.add(node); + } + } + supported = supportedBuilder.build(); + unsupported = unsupportedBuilder.build(); + } + + @Override + public List getNotSupported() { + maybeInitialize(); + return checkStateNotNull(unsupported); + } + + @Override + public int numSupported() { + maybeInitialize(); + return BeamSqlTableFilter.expressionsInFilter(checkStateNotNull(supported)); + } + + public List getSupported() { + maybeInitialize(); + return checkStateNotNull(supported); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(IcebergFilter.class) + .add( + "supported", + checkStateNotNull(supported).stream() + .map(RexNode::toString) + .collect(Collectors.joining())) + .add( + "unsupported", + checkStateNotNull(unsupported).stream() + .map(RexNode::toString) + .collect(Collectors.joining())) + .toString(); + } + + /** + * Check whether a {@code RexNode} is supported. As of right now Iceberg supports: 1. Complex + * predicates (both conjunction and disjunction). 2. Comparison between a column and a literal. + * + * @param node A node to check for predicate push-down support. + * @return A pair containing a boolean whether an expression is supported and the number of input + * references used by the expression. + */ + private Pair isSupported(RexNode node) { + int numberOfInputRefs = 0; + boolean isSupported = true; + + if (node instanceof RexCall) { + RexCall compositeNode = (RexCall) node; + if (!SUPPORTED_OPS.contains(node.getKind())) { + isSupported = false; + } else { + for (RexNode operand : compositeNode.getOperands()) { + // All operands must be supported for a parent node to be supported. + Pair childSupported = isSupported(operand); + if (!node.getKind().belongsTo(ImmutableSet.of(AND, OR))) { + numberOfInputRefs += childSupported.getRight(); + } + // Predicate functions with multiple columns are unsupported. + isSupported = numberOfInputRefs < 2 && childSupported.getLeft(); + } + } + } else if (node instanceof RexInputRef) { + numberOfInputRefs = 1; + } else if (node instanceof RexLiteral) { + // RexLiterals are expected, but no action is needed. + } else { + throw new UnsupportedOperationException( + "Encountered an unexpected node type: " + node.getClass().getSimpleName()); + } + + return Pair.of(isSupported, numberOfInputRefs); + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java index 83dfe9959207..9a87edff2a21 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTable.java @@ -18,22 +18,42 @@ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import com.fasterxml.jackson.databind.node.ObjectNode; +import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.function.IntFunction; +import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter; +import org.apache.beam.sdk.extensions.sql.meta.DefaultTableFilter; +import org.apache.beam.sdk.extensions.sql.meta.ProjectSupport; import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable; import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BeamSqlUnparseContext; import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; import org.apache.beam.sdk.managed.Managed; +import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rel.rel2sql.SqlImplementor; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlNode; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.dialect.BigQuerySqlDialect; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.parser.SqlParserPos; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class IcebergTable extends SchemaBaseBeamTable { + private static final Logger LOG = LoggerFactory.getLogger(IcebergTable.class); @VisibleForTesting static final String CATALOG_PROPERTIES_FIELD = "catalog_properties"; @VisibleForTesting static final String HADOOP_CONFIG_PROPERTIES_FIELD = "config_properties"; @VisibleForTesting static final String CATALOG_NAME_FIELD = "catalog_name"; @@ -56,6 +76,16 @@ class IcebergTable extends SchemaBaseBeamTable { } } + @Override + public POutput buildIOWriter(PCollection input) { + ImmutableMap.Builder configBuilder = ImmutableMap.builder(); + configBuilder.putAll(getBaseConfig()); + if (triggeringFrequency != null) { + configBuilder.put(TRIGGERING_FREQUENCY_FIELD, triggeringFrequency); + } + return input.apply(Managed.write(Managed.ICEBERG).withConfig(configBuilder.build())); + } + @Override public PCollection buildIOReader(PBegin begin) { return begin @@ -64,13 +94,44 @@ public PCollection buildIOReader(PBegin begin) { } @Override - public POutput buildIOWriter(PCollection input) { - ImmutableMap.Builder configBuilder = ImmutableMap.builder(); - configBuilder.putAll(getBaseConfig()); - if (triggeringFrequency != null) { - configBuilder.put(TRIGGERING_FREQUENCY_FIELD, triggeringFrequency); + public PCollection buildIOReader( + PBegin begin, BeamSqlTableFilter filter, List fieldNames) { + + Map readConfig = new HashMap<>(getBaseConfig()); + + if (!(filter instanceof DefaultTableFilter)) { + IcebergFilter icebergFilter = (IcebergFilter) filter; + if (!icebergFilter.getSupported().isEmpty()) { + String expression = generateFilterExpression(getSchema(), icebergFilter.getSupported()); + if (!expression.isEmpty()) { + LOG.info("Pushing down the following filter: {}", expression); + readConfig.put("filter", expression); + } + } } - return input.apply(Managed.write(Managed.ICEBERG).withConfig(configBuilder.build())); + + if (!fieldNames.isEmpty()) { + readConfig.put("keep", fieldNames); + } + + return begin + .apply("Read Iceberg with push-down", Managed.read(Managed.ICEBERG).withConfig(readConfig)) + .getSinglePCollection(); + } + + @Override + public ProjectSupport supportsProjects() { + return ProjectSupport.WITHOUT_FIELD_REORDERING; + } + + @Override + public BeamSqlTableFilter constructFilter(List filter) { + return new IcebergFilter(filter); + } + + @Override + public PCollection.IsBounded isBounded() { + return PCollection.IsBounded.BOUNDED; } private Map getBaseConfig() { @@ -91,8 +152,25 @@ private Map getBaseConfig() { return managedConfigBuilder.build(); } - @Override - public PCollection.IsBounded isBounded() { - return PCollection.IsBounded.BOUNDED; + private String generateFilterExpression(Schema schema, List supported) { + final IntFunction field = + i -> new SqlIdentifier(schema.getField(i).getName(), SqlParserPos.ZERO); + + SqlImplementor.Context context = new BeamSqlUnparseContext(field); + + // Create a single SqlNode from a list of RexNodes + SqlNode andSqlNode = null; + for (RexNode node : supported) { + SqlNode sqlNode = context.toSql(null, node); + if (andSqlNode == null) { + andSqlNode = sqlNode; + continue; + } + // AND operator must have exactly 2 operands. + andSqlNode = + SqlStdOperatorTable.AND.createCall( + SqlParserPos.ZERO, ImmutableList.of(andSqlNode, sqlNode)); + } + return checkStateNotNull(andSqlNode).toSqlString(BigQuerySqlDialect.DEFAULT).getSql(); } } diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java new file mode 100644 index 000000000000..faa86e1c257a --- /dev/null +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java @@ -0,0 +1,122 @@ +/* + * 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.beam.sdk.extensions.sql.meta.provider.iceberg; + +import static org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider.PUSH_DOWN_OPTION; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; + +import org.apache.beam.sdk.extensions.sql.TableUtils; +import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; +import org.apache.beam.sdk.extensions.sql.meta.Table; +import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider; +import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider.PushDownOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.testcontainers.shaded.org.apache.commons.lang3.tuple.Pair; + +@RunWith(JUnit4.class) +public class IcebergFilterTest { + // TODO: add date, time, and datetime fields. + private static final Schema BASIC_SCHEMA = + Schema.builder() + .addInt32Field("unused1") + .addInt32Field("id") + .addStringField("name") + .addInt16Field("unused2") + .addBooleanField("b") + .build(); + + private BeamSqlEnv sqlEnv; + + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Before + public void buildUp() { + TestTableProvider tableProvider = new TestTableProvider(); + Table table = getTable("TEST", PushDownOptions.NONE); + tableProvider.createTable(table); + tableProvider.addRows( + table.getName(), + row(BASIC_SCHEMA, 100, 1, "one", (short) 100, true), + row(BASIC_SCHEMA, 200, 2, "two", (short) 200, false)); + + sqlEnv = + BeamSqlEnv.builder(tableProvider) + .setPipelineOptions(PipelineOptionsFactory.create()) + .build(); + } + + @Test + public void testIsSupported() { + ImmutableList> sqlQueries = + ImmutableList.of( + Pair.of("select * from TEST where unused1=100", true), + Pair.of("select * from TEST where unused1 in (100, 200)", true), + Pair.of("select * from TEST where unused1+10=110", true), + Pair.of("select * from TEST where b", true), + Pair.of( + "select * from TEST where unused1>100 and unused1<=200 and id<>1 and (name='two' or id=2)", + true), + Pair.of("select * from TEST where unused2=200", true), + Pair.of("select * from TEST where name like 'o%e'", true), + // Functions involving more than one column are not supported yet. + Pair.of("select * from TEST where unused1=unused2 and id=2", false), + Pair.of("select * from TEST where unused1+unused2=10", false)); + + for (Pair query : sqlQueries) { + String sql = query.getLeft(); + Boolean isSupported = query.getRight(); + + BeamRelNode beamRelNode = sqlEnv.parseQuery(sql); + assertThat(beamRelNode, instanceOf(BeamCalcRel.class)); + IcebergFilter filter = + new IcebergFilter(((BeamCalcRel) beamRelNode).getProgram().split().right); + + assertThat( + "Query: '" + sql + "' is expected to be " + (isSupported ? "supported." : "unsupported."), + filter.getNotSupported().isEmpty() == isSupported); + } + } + + private static Table getTable(String name, PushDownOptions options) { + return Table.builder() + .name(name) + .comment(name + " table") + .schema(BASIC_SCHEMA) + .properties( + TableUtils.parseProperties( + "{ " + PUSH_DOWN_OPTION + ": " + "\"" + options.toString() + "\" }")) + .type("test") + .build(); + } + + private static Row row(Schema schema, Object... objects) { + return Row.withSchema(schema).addValues(objects).build(); + } +} diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index 0cfe64e8c13b..08b962d9b341 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -28,7 +28,9 @@ import static org.apache.beam.sdk.schemas.Schema.FieldType.array; import static org.apache.beam.sdk.schemas.Schema.FieldType.row; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertEquals; import com.google.api.services.bigquery.model.TableRow; @@ -38,6 +40,8 @@ import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamPushDownIOSourceRel; +import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; @@ -48,10 +52,12 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.joda.time.Duration; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.TestName; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -79,6 +85,8 @@ public class IcebergReadWriteIT { @Rule public transient TestPipeline writePipeline = TestPipeline.create(); @Rule public transient TestPipeline readPipeline = TestPipeline.create(); + @Rule public TestName testName = new TestName(); + private static IcebergTableProvider provider; private static final BigqueryClient BQ_CLIENT = new BigqueryClient("IcebergReadWriteIT"); static final String DATASET = "iceberg_sql_tests_" + System.nanoTime(); @@ -94,6 +102,15 @@ public static void createDataset() throws IOException, InterruptedException { TestPipeline.testingPipelineOptions().getTempLocation(), IcebergReadWriteIT.class.getSimpleName(), UUID.randomUUID()); + provider = + IcebergTableProvider.create() + .withCatalogProperties( + ImmutableMap.of( + "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", + "warehouse", warehouse, + "gcp_project", OPTIONS.getProject(), + "gcp_region", "us-central1")); BQ_CLIENT.createNewDataset(OPTIONS.getProject(), DATASET); } @@ -104,17 +121,8 @@ public static void deleteDataset() { @Test public void testSqlWriteAndRead() throws IOException, InterruptedException { - BeamSqlEnv sqlEnv = - BeamSqlEnv.inMemory( - IcebergTableProvider.create() - .withCatalogProperties( - ImmutableMap.of( - "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", - "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", - "warehouse", warehouse, - "gcp_project", OPTIONS.getProject(), - "gcp_region", "us-central1"))); - String tableIdentifier = DATASET + ".my_table"; + BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(provider); + String tableIdentifier = DATASET + "." + testName.getMethodName(); // 1) create beam table String createTableStatement = @@ -153,7 +161,6 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { + "ROW(ARRAY['foo', 'bar'], 456), " + "ROW(ARRAY['cat', 'dog'], 789)]" + ")"; - sqlEnv.parseQuery(insertStatement); BeamSqlRelUtils.toPCollection(writePipeline, sqlEnv.parseQuery(insertStatement)); writePipeline.run().waitUntilFinish(); @@ -190,8 +197,60 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { assertThat(state, equalTo(PipelineResult.State.DONE)); } + @Test + public void testSQLReadWithProjectAndFilterPushDown() { + BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(provider); + String tableIdentifier = DATASET + "." + testName.getMethodName(); + + String createTableStatement = + "CREATE EXTERNAL TABLE TEST( \n" + + " c_integer INTEGER, \n" + + " c_float FLOAT, \n" + + " c_boolean BOOLEAN, \n" + + " c_timestamp TIMESTAMP, \n" + + " c_varchar VARCHAR \n " + + ") \n" + + "TYPE 'iceberg' \n" + + "LOCATION '" + + tableIdentifier + + "'"; + sqlEnv.executeDdl(createTableStatement); + + String insertStatement = + "INSERT INTO TEST VALUES " + + "(123, 1.23, TRUE, TIMESTAMP '2025-05-22 20:17:40.123', 'a'), " + + "(456, 4.56, FALSE, TIMESTAMP '2025-05-25 20:17:40.123', 'b'), " + + "(789, 7.89, TRUE, TIMESTAMP '2025-05-28 20:17:40.123', 'c')"; + BeamSqlRelUtils.toPCollection(writePipeline, sqlEnv.parseQuery(insertStatement)); + writePipeline.run().waitUntilFinish(Duration.standardMinutes(5)); + + String selectTableStatement = + "SELECT c_integer, c_varchar FROM TEST where " + + "(c_boolean=TRUE and c_varchar in ('a', 'b')) or c_float > 5"; + BeamRelNode relNode = sqlEnv.parseQuery(selectTableStatement); + PCollection output = BeamSqlRelUtils.toPCollection(readPipeline, relNode); + + assertThat(relNode, instanceOf(BeamPushDownIOSourceRel.class)); + // Unused fields should not be projected by an IO + assertThat(relNode.getRowType().getFieldNames(), containsInAnyOrder("c_integer", "c_varchar")); + + assertThat( + output.getSchema(), + equalTo( + Schema.builder() + .addNullableField("c_integer", INT32) + .addNullableField("c_varchar", STRING) + .build())); + + PAssert.that(output) + .containsInAnyOrder( + Row.withSchema(output.getSchema()).addValues(123, "a").build(), + Row.withSchema(output.getSchema()).addValues(789, "c").build()); + PipelineResult.State state = readPipeline.run().waitUntilFinish(Duration.standardMinutes(5)); + assertThat(state, equalTo(PipelineResult.State.DONE)); + } + private Row nestedRow(List arr, Integer intVal) { - System.out.println(arr); return Row.withSchema(NESTED_SCHEMA).addValues(arr, intVal).build(); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java index 74ff86fcafda..35d92dc39eae 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java @@ -24,11 +24,14 @@ import java.time.LocalDate; import java.time.LocalDateTime; import java.time.LocalTime; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.function.BiFunction; import java.util.stream.Collectors; +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlBasicCall; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlIdentifier; import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; @@ -45,6 +48,7 @@ import org.apache.iceberg.expressions.Expression.Operation; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.NaNUtil; import org.checkerframework.checker.nullness.qual.Nullable; @@ -55,8 +59,9 @@ * *

Note: Only supports top-level fields (i.e. cannot reference nested fields). */ -class FilterUtils { - private static final Map FILTERS = +@Internal +public class FilterUtils { + static final Map FILTERS = ImmutableMap.builder() .put(SqlKind.IS_NULL, Operation.IS_NULL) .put(SqlKind.IS_NOT_NULL, Operation.NOT_NULL) @@ -72,6 +77,57 @@ class FilterUtils { .put(SqlKind.OR, Operation.OR) .build(); + public static Set SUPPORTED_OPS = FILTERS.keySet(); + + /** + * Parses a SQL filter expression string and returns a set of all field names referenced within + * it. + */ + static Set getReferencedFieldNames(@Nullable String filter) { + if (filter == null || filter.trim().isEmpty()) { + return new HashSet<>(); + } + + SqlParser parser = SqlParser.create(filter); + try { + SqlNode expression = parser.parseExpression(); + Set fieldNames = new HashSet<>(); + extractFieldNames(expression, fieldNames); + System.out.println("xxx fields in filter: " + fieldNames); + return fieldNames; + } catch (Exception exception) { + throw new RuntimeException( + String.format("Encountered an error when parsing filter: '%s'", filter), exception); + } + } + + private static void extractFieldNames(SqlNode node, Set fieldNames) { + if (node instanceof SqlIdentifier) { + fieldNames.add(((SqlIdentifier) node).getSimple()); + } else if (node instanceof SqlBasicCall) { + // recursively check operands + SqlBasicCall call = (SqlBasicCall) node; + for (SqlNode operand : call.getOperandList()) { + extractFieldNames(operand, fieldNames); + } + } else if (node instanceof SqlNodeList) { + // For IN clauses, the right-hand side is a SqlNodeList, so iterate through its elements + SqlNodeList nodeList = (SqlNodeList) node; + for (SqlNode element : nodeList.getList()) { + if (element != null) { + extractFieldNames(element, fieldNames); + } + } + } + // SqlLiteral nodes do not contain field names, so we can ignore them. + } + /** + * parses a SQL filter expression string into an Iceberg {@link Expression} that can be used for + * data pruning. + * + *

Note: This utility currently supports only top-level fields within the filter expression. + * Nested field references are not supported. + */ static Expression convert(@Nullable String filter, Schema schema) { if (filter == null) { return Expressions.alwaysTrue(); @@ -210,8 +266,10 @@ private static Expression convertFieldInLiteral(Operation op, SqlBasicCall call, checkArgument( value instanceof SqlNodeList, "Expected right hand side to be a list but got " + value.getClass()); - String name = ((SqlIdentifier) term).getSimple(); - TypeID type = schema.findType(name).typeId(); + String caseInsensitiveName = ((SqlIdentifier) term).getSimple(); + Types.NestedField field = schema.caseInsensitiveFindField(caseInsensitiveName); + String name = field.name(); + TypeID type = field.type().typeId(); List list = ((SqlNodeList) value) .getList().stream().filter(Objects::nonNull).collect(Collectors.toList()); @@ -236,13 +294,17 @@ private static Expression convertFieldAndLiteral( SqlNode left = getLeftChild(call); SqlNode right = getRightChild(call); if (left instanceof SqlIdentifier && right instanceof SqlLiteral) { - String name = ((SqlIdentifier) left).getSimple(); - TypeID type = schema.findType(name).typeId(); + String caseInsensitiveName = ((SqlIdentifier) left).getSimple(); + Types.NestedField field = schema.caseInsensitiveFindField(caseInsensitiveName); + String name = field.name(); + TypeID type = field.type().typeId(); Object value = convertLiteral((SqlLiteral) right, name, type); return convertLR.apply(name, value); } else if (left instanceof SqlLiteral && right instanceof SqlIdentifier) { - String name = ((SqlIdentifier) right).getSimple(); - TypeID type = schema.findType(name).typeId(); + String caseInsensitiveName = ((SqlIdentifier) right).getSimple(); + Types.NestedField field = schema.caseInsensitiveFindField(caseInsensitiveName); + String name = field.name(); + TypeID type = field.type().typeId(); Object value = convertLiteral((SqlLiteral) left, name, type); return convertRL.apply(name, value); } else { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java index 6b2ea6721e31..956e45651df7 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergIO.java @@ -603,7 +603,7 @@ public PCollection expand(PBegin input) { .setUseCdc(getUseCdc()) .setKeepFields(getKeep()) .setDropFields(getDrop()) - .setFilter(FilterUtils.convert(getFilter(), table.schema())) + .setFilterString(getFilter()) .build(); scanConfig.validate(table); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java index 4edf3512952e..68bffc75e80c 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -75,19 +76,57 @@ public Table getTable() { @VisibleForTesting static org.apache.iceberg.Schema resolveSchema( org.apache.iceberg.Schema schema, @Nullable List keep, @Nullable List drop) { + return resolveSchema(schema, keep, drop, null); + } + + @VisibleForTesting + static org.apache.iceberg.Schema resolveSchema( + org.apache.iceberg.Schema schema, + @Nullable List keep, + @Nullable List drop, + @Nullable Set fieldsInFilter) { + ImmutableList.Builder selectedFields = ImmutableList.builder(); if (keep != null && !keep.isEmpty()) { - schema = schema.select(keep); + selectedFields.addAll(keep); } else if (drop != null && !drop.isEmpty()) { Set fields = schema.columns().stream().map(Types.NestedField::name).collect(Collectors.toSet()); drop.forEach(fields::remove); - schema = schema.select(fields); + selectedFields.addAll(fields); } - return schema; + + if (fieldsInFilter != null && !fieldsInFilter.isEmpty()) { + fieldsInFilter.stream() + .map(f -> schema.caseInsensitiveFindField(f).name()) + .forEach(selectedFields::add); + } + return schema.select(selectedFields.build()); } + private org.apache.iceberg.@Nullable Schema cachedProjectedSchema; + /** Returns the projected Schema after applying column pruning. */ public org.apache.iceberg.Schema getProjectedSchema() { - return resolveSchema(getTable().schema(), getKeepFields(), getDropFields()); + if (cachedProjectedSchema == null) { + cachedProjectedSchema = resolveSchema(getTable().schema(), getKeepFields(), getDropFields()); + } + return cachedProjectedSchema; + } + + private org.apache.iceberg.@Nullable Schema cachedRequiredSchema; + /** + * Returns a Schema that includes explicitly selected fields and fields referenced in the filter + * statement. + */ + public org.apache.iceberg.Schema getRequiredSchema() { + if (cachedRequiredSchema == null) { + cachedRequiredSchema = + resolveSchema( + getTable().schema(), + getKeepFields(), + getDropFields(), + FilterUtils.getReferencedFieldNames(getFilterString())); + } + return cachedRequiredSchema; } @Pure @@ -98,7 +137,7 @@ public Evaluator getEvaluator() { return null; } if (cachedEvaluator == null) { - cachedEvaluator = new Evaluator(getProjectedSchema().asStruct(), filter); + cachedEvaluator = new Evaluator(getRequiredSchema().asStruct(), filter); } return cachedEvaluator; } @@ -106,7 +145,18 @@ public Evaluator getEvaluator() { private transient @Nullable Evaluator cachedEvaluator; @Pure - public abstract @Nullable Expression getFilter(); + @Nullable + public Expression getFilter() { + if (cachedFilter == null) { + cachedFilter = FilterUtils.convert(getFilterString(), getTable().schema()); + } + return cachedFilter; + } + + private transient @Nullable Expression cachedFilter; + + @Pure + public abstract @Nullable String getFilterString(); @Pure public abstract @Nullable Boolean getCaseSensitive(); @@ -172,7 +222,7 @@ public Evaluator getEvaluator() { public static Builder builder() { return new AutoValue_IcebergScanConfig.Builder() .setScanType(ScanType.TABLE) - .setFilter(null) + .setFilterString(null) .setCaseSensitive(null) .setOptions(ImmutableMap.of()) .setSnapshot(null) @@ -211,7 +261,7 @@ public Builder setTableIdentifier(String... names) { public abstract Builder setSchema(Schema schema); - public abstract Builder setFilter(@Nullable Expression filter); + public abstract Builder setFilterString(@Nullable String filterString); public abstract Builder setCaseSensitive(@Nullable Boolean caseSensitive); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index 366f5565d425..528b89c203bf 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -74,10 +74,9 @@ public void process( return; } FileScanTask task = fileScanTasks.get((int) l); - org.apache.iceberg.Schema projected = scanConfig.getProjectedSchema(); - Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(projected); + Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(scanConfig.getProjectedSchema()); try (CloseableIterable fullIterable = - ReadUtils.createReader(task, table, projected)) { + ReadUtils.createReader(task, table, scanConfig.getRequiredSchema())) { CloseableIterable reader = ReadUtils.maybeApplyFilter(fullIterable, scanConfig); for (Record record : reader) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java index c52b39dde1c2..81ec229df70f 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskReader.java @@ -50,7 +50,6 @@ import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; -import org.checkerframework.checker.nullness.qual.NonNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -58,7 +57,6 @@ class ScanTaskReader extends BoundedSource.BoundedReader { private static final Logger LOG = LoggerFactory.getLogger(ScanTaskReader.class); private final ScanTaskSource source; - private final org.apache.iceberg.Schema project; private final Schema beamSchema; transient @Nullable FileIO io; @@ -69,8 +67,7 @@ class ScanTaskReader extends BoundedSource.BoundedReader { public ScanTaskReader(ScanTaskSource source) { this.source = source; - this.project = source.getSchema(); - this.beamSchema = icebergSchemaToBeamSchema(project); + this.beamSchema = icebergSchemaToBeamSchema(source.getScanConfig().getProjectedSchema()); } @Override @@ -96,8 +93,7 @@ public boolean advance() throws IOException { // This nullness annotation is incorrect, but the most expedient way to work with Iceberg's APIs // which are not null-safe. - @SuppressWarnings("nullness") - org.apache.iceberg.@NonNull Schema project = this.project; + org.apache.iceberg.Schema requiredSchema = source.getScanConfig().getRequiredSchema(); @Nullable String nameMapping = source.getTable().properties().get(TableProperties.DEFAULT_NAME_MAPPING); @@ -125,7 +121,8 @@ public boolean advance() throws IOException { DataFile file = fileTask.file(); InputFile input = decryptor.getInputFile(fileTask); Map idToConstants = - ReadUtils.constantsMap(fileTask, IdentityPartitionConverters::convertConstant, project); + ReadUtils.constantsMap( + fileTask, IdentityPartitionConverters::convertConstant, requiredSchema); CloseableIterable iterable; switch (file.format()) { @@ -134,10 +131,10 @@ public boolean advance() throws IOException { ORC.ReadBuilder orcReader = ORC.read(input) .split(fileTask.start(), fileTask.length()) - .project(project) + .project(requiredSchema) .createReaderFunc( fileSchema -> - GenericOrcReader.buildReader(project, fileSchema, idToConstants)) + GenericOrcReader.buildReader(requiredSchema, fileSchema, idToConstants)) .filter(fileTask.residual()); if (nameMapping != null) { @@ -151,10 +148,11 @@ public boolean advance() throws IOException { Parquet.ReadBuilder parquetReader = Parquet.read(input) .split(fileTask.start(), fileTask.length()) - .project(project) + .project(requiredSchema) .createReaderFunc( fileSchema -> - GenericParquetReaders.buildReader(project, fileSchema, idToConstants)) + GenericParquetReaders.buildReader( + requiredSchema, fileSchema, idToConstants)) .filter(fileTask.residual()); if (nameMapping != null) { @@ -168,9 +166,9 @@ public boolean advance() throws IOException { Avro.ReadBuilder avroReader = Avro.read(input) .split(fileTask.start(), fileTask.length()) - .project(project) + .project(requiredSchema) .createReaderFunc( - fileSchema -> DataReader.create(project, fileSchema, idToConstants)); + fileSchema -> DataReader.create(requiredSchema, fileSchema, idToConstants)); if (nameMapping != null) { avroReader.withNameMapping(NameMappingParser.fromJson(nameMapping)); @@ -182,7 +180,8 @@ public boolean advance() throws IOException { throw new UnsupportedOperationException("Cannot read format: " + file.format()); } GenericDeleteFilter deleteFilter = - new GenericDeleteFilter(checkStateNotNull(io), fileTask, fileTask.schema(), project); + new GenericDeleteFilter( + checkStateNotNull(io), fileTask, fileTask.schema(), requiredSchema); iterable = deleteFilter.filter(iterable); iterable = ReadUtils.maybeApplyFilter(iterable, source.getScanConfig()); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskSource.java index 2c92c5572c6d..7eb5730a61ec 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskSource.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ScanTaskSource.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.checkerframework.dataflow.qual.Pure; @@ -54,11 +53,6 @@ CombinedScanTask getTask() { return task; } - @Pure - Schema getSchema() { - return scanConfig.getProjectedSchema(); - } - @Pure IcebergScanConfig getScanConfig() { return scanConfig; diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FilterUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FilterUtilsTest.java index 34e7be619110..ff6383d4d046 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FilterUtilsTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/FilterUtilsTest.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.iceberg; import static org.apache.beam.sdk.io.iceberg.FilterUtils.convert; +import static org.apache.beam.sdk.io.iceberg.FilterUtils.getReferencedFieldNames; import static org.apache.iceberg.expressions.Expressions.and; import static org.apache.iceberg.expressions.Expressions.equal; import static org.apache.iceberg.expressions.Expressions.greaterThan; @@ -46,11 +47,13 @@ import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; @@ -508,6 +511,22 @@ public void testScanFiles() throws IOException { assertEquals(expectedFiles, actualFiles.build()); } + @Test + public void testReferencedFieldsInFilter() { + List>> cases = + Arrays.asList( + Pair.of("field_1 < 35", Sets.newHashSet("FIELD_1")), + Pair.of("\"field_1\" in (1, 2, 3)", Sets.newHashSet("field_1")), + Pair.of("field_1 < 35 and \"fiELd_2\" = TRUE", Sets.newHashSet("FIELD_1", "fiELd_2")), + Pair.of( + "(\"field_1\" < 35 and \"field_2\" = TRUE) or \"field_3\" in ('a', 'b')", + Sets.newHashSet("field_1", "field_2", "field_3"))); + + for (Pair> pair : cases) { + assertEquals(pair.getRight(), getReferencedFieldNames(pair.getLeft())); + } + } + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java index 5bfcb1345c37..bb303ea9c305 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergIOReadTest.java @@ -32,6 +32,7 @@ import java.io.File; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -357,7 +358,6 @@ public void testScanWithFilter() throws Exception { TableIdentifier tableId = TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); - final Schema schema = icebergSchemaToBeamSchema(TestFixtures.SCHEMA); List> expectedRecords = warehouse.commitData(simpleTable); @@ -365,24 +365,26 @@ public void testScanWithFilter() throws Exception { IcebergIO.readRows(catalogConfig()) .from(tableId) .withFilter( - "\"id\" < 10 AND \"id\" >= 2 AND \"data\" <> 'clammy' AND \"data\" <> 'brainy'"); + "\"id\" < 10 AND \"id\" >= 2 AND \"data\" <> 'clammy' AND \"data\" <> 'brainy'") + .keeping(Arrays.asList("id")); if (useIncrementalScan) { read = read.withCdc().toSnapshot(simpleTable.currentSnapshot().snapshotId()); } + final Schema outputSchema = icebergSchemaToBeamSchema(TestFixtures.SCHEMA.select("id")); final List expectedRows = expectedRecords.stream() .flatMap(List::stream) - .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) .filter( - row -> { - long id = checkStateNotNull(row.getInt64("id")); - String data = checkStateNotNull(row.getString("data")); + record -> { + long id = checkStateNotNull((Long) record.getField("id")); + String data = checkStateNotNull((String) record.getField("data")); return id < 10 && id >= 2 && !Objects.equals(data, "clammy") && !Objects.equals(data, "brainy"); }) + .map(record -> IcebergUtils.icebergRecordToBeamRow(outputSchema, record)) .collect(Collectors.toList()); PCollection output = testPipeline.apply(read).apply(new PrintRow()); From 867aaf1c4b598c8844eec423462248042baf7fa2 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 28 May 2025 19:12:41 -0400 Subject: [PATCH 13/20] spotless --- .../extensions/sql/meta/provider/iceberg/IcebergFilterTest.java | 2 +- .../main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java index faa86e1c257a..3424504459da 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java @@ -32,13 +32,13 @@ import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.calcite.v1_28_0.org.apache.commons.lang3.tuple.Pair; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.testcontainers.shaded.org.apache.commons.lang3.tuple.Pair; @RunWith(JUnit4.class) public class IcebergFilterTest { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java index 35d92dc39eae..c70425eb0ad5 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/FilterUtils.java @@ -77,7 +77,7 @@ public class FilterUtils { .put(SqlKind.OR, Operation.OR) .build(); - public static Set SUPPORTED_OPS = FILTERS.keySet(); + public static final Set SUPPORTED_OPS = FILTERS.keySet(); /** * Parses a SQL filter expression string and returns a set of all field names referenced within From 841a3f7d8d55acb3f7e792eeebeafc4f17b7017e Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 29 May 2025 13:29:13 -0400 Subject: [PATCH 14/20] fixes --- .../apache/beam/sdk/io/iceberg/IcebergScanConfig.java | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java index 68bffc75e80c..4f46ec091099 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java @@ -85,22 +85,23 @@ static org.apache.iceberg.Schema resolveSchema( @Nullable List keep, @Nullable List drop, @Nullable Set fieldsInFilter) { - ImmutableList.Builder selectedFields = ImmutableList.builder(); + ImmutableList.Builder selectedFieldsBuilder = ImmutableList.builder(); if (keep != null && !keep.isEmpty()) { - selectedFields.addAll(keep); + selectedFieldsBuilder.addAll(keep); } else if (drop != null && !drop.isEmpty()) { Set fields = schema.columns().stream().map(Types.NestedField::name).collect(Collectors.toSet()); drop.forEach(fields::remove); - selectedFields.addAll(fields); + selectedFieldsBuilder.addAll(fields); } if (fieldsInFilter != null && !fieldsInFilter.isEmpty()) { fieldsInFilter.stream() .map(f -> schema.caseInsensitiveFindField(f).name()) - .forEach(selectedFields::add); + .forEach(selectedFieldsBuilder::add); } - return schema.select(selectedFields.build()); + ImmutableList selectedFields = selectedFieldsBuilder.build(); + return selectedFields.isEmpty() ? schema : schema.select(selectedFields); } private org.apache.iceberg.@Nullable Schema cachedProjectedSchema; From bcb3d103591908191cb2120b0e3ea8103fdb2704 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 29 May 2025 14:05:21 -0400 Subject: [PATCH 15/20] fixes --- .../sql/meta/provider/iceberg/IcebergFilterTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java index 3424504459da..f14344b4f1fe 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergFilterTest.java @@ -40,6 +40,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +/** Test class for {@link IcebergFilter}. */ @RunWith(JUnit4.class) public class IcebergFilterTest { // TODO: add date, time, and datetime fields. @@ -84,10 +85,8 @@ public void testIsSupported() { "select * from TEST where unused1>100 and unused1<=200 and id<>1 and (name='two' or id=2)", true), Pair.of("select * from TEST where unused2=200", true), - Pair.of("select * from TEST where name like 'o%e'", true), // Functions involving more than one column are not supported yet. - Pair.of("select * from TEST where unused1=unused2 and id=2", false), - Pair.of("select * from TEST where unused1+unused2=10", false)); + Pair.of("select * from TEST where unused1=unused2 and id=2", false)); for (Pair query : sqlQueries) { String sql = query.getLeft(); From f447a7c806c2f4757873cfb6e6dc1d96e8ce4fb6 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 5 Jun 2025 10:34:51 -0400 Subject: [PATCH 16/20] updates --- .../iceberg/IcebergTableProvider.java | 34 +++++++++++++------ .../sdk/extensions/sql/PubsubToIcebergIT.java | 25 +++++++------- .../provider/iceberg/IcebergReadWriteIT.java | 23 ++++++------- .../iceberg/IcebergTableProviderTest.java | 19 +++++------ 4 files changed, 55 insertions(+), 46 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java index 575d55044ee8..49395922af7b 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java @@ -23,27 +23,41 @@ import org.apache.beam.sdk.extensions.sql.meta.provider.InMemoryMetaTableProvider; import org.apache.beam.sdk.io.iceberg.IcebergCatalogConfig; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; public class IcebergTableProvider extends InMemoryMetaTableProvider { + private static final String BEAM_HADOOP_PREFIX = "beam.catalog.%s.hadoop"; @VisibleForTesting IcebergCatalogConfig catalogConfig = IcebergCatalogConfig.builder().build(); public static IcebergTableProvider create() { return new IcebergTableProvider(); } - public IcebergTableProvider withCatalogProperties(Map catalogProperties) { - catalogConfig = catalogConfig.toBuilder().setCatalogProperties(catalogProperties).build(); - return this; + static IcebergTableProvider create(String name, Map properties) { + IcebergTableProvider provider = new IcebergTableProvider(); + provider.initialize(name, properties); + return provider; } - public IcebergTableProvider withHadoopConfProperties(Map hadoopConfProperties) { - catalogConfig = catalogConfig.toBuilder().setConfigProperties(hadoopConfProperties).build(); - return this; - } + public void initialize(String name, Map properties) { + ImmutableMap.Builder catalogProps = ImmutableMap.builder(); + ImmutableMap.Builder hadoopProps = ImmutableMap.builder(); + String hadoopPrefix = String.format(BEAM_HADOOP_PREFIX, name); + + for (Map.Entry entry : properties.entrySet()) { + if (entry.getKey().startsWith(hadoopPrefix)) { + hadoopProps.put(entry.getKey(), entry.getValue()); + } else { + catalogProps.put(entry.getKey(), entry.getValue()); + } + } - public IcebergTableProvider withCatalogName(String name) { - catalogConfig = catalogConfig.toBuilder().setCatalogName(name).build(); - return this; + catalogConfig = + IcebergCatalogConfig.builder() + .setCatalogName(name) + .setCatalogProperties(catalogProps.build()) + .setConfigProperties(hadoopProps.build()) + .build(); } @Override diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java index 1fd619a684c4..76e005c3d61a 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java @@ -62,7 +62,7 @@ public class PubsubToIcebergIT implements Serializable { @Rule public transient TestPipeline pipeline = TestPipeline.create(); @Rule public transient TestPubsub pubsub = TestPubsub.create(); - + private static final IcebergTableProvider PROVIDER = IcebergTableProvider.create(); private static final BigqueryClient BQ_CLIENT = new BigqueryClient("PubsubToIcebergIT"); static final String DATASET = "sql_pubsub_to_iceberg_it_" + System.nanoTime(); static String warehouse; @@ -78,22 +78,23 @@ public static void createDataset() throws IOException, InterruptedException { TestPipeline.testingPipelineOptions().getTempLocation(), PubsubToIcebergIT.class.getSimpleName(), UUID.randomUUID()); + icebergConfig = + ImmutableMap.of( + "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", + "warehouse", warehouse, + "gcp_project", OPTIONS.getProject(), + "gcp_region", "us-central1"); + PROVIDER.initialize(PubsubToIcebergIT.class.getSimpleName(), icebergConfig); BQ_CLIENT.createNewDataset(OPTIONS.getProject(), DATASET); } private String tableIdentifier; - private Map icebergConfig; + private static Map icebergConfig; @Before public void setup() { tableIdentifier = DATASET + "." + testName.getMethodName(); - icebergConfig = - ImmutableMap.of( - "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", - "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", - "warehouse", warehouse, - "gcp_project", OPTIONS.getProject(), - "gcp_region", "us-central1"); } @AfterClass @@ -137,8 +138,7 @@ public void testSimpleInsert() throws Exception { SqlTransform.query(insertStatement) .withDdlString(pubsubTableString) .withDdlString(icebergTableString) - .withTableProvider( - "iceberg", IcebergTableProvider.create().withCatalogProperties(icebergConfig))); + .withTableProvider("iceberg", PROVIDER)); pipeline.run(); // Block until a subscription for this topic exists @@ -187,8 +187,7 @@ public void testSimpleInsertFlat() throws Exception { SqlTransform.query(insertStatement) .withDdlString(pubsubTableString) .withDdlString(bqTableString) - .withTableProvider( - "iceberg", IcebergTableProvider.create().withCatalogProperties(icebergConfig))); + .withTableProvider("iceberg", PROVIDER)); pipeline.run(); validateRowsWritten(); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index 08b962d9b341..9a5749c26032 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -86,7 +86,7 @@ public class IcebergReadWriteIT { @Rule public transient TestPipeline writePipeline = TestPipeline.create(); @Rule public transient TestPipeline readPipeline = TestPipeline.create(); @Rule public TestName testName = new TestName(); - private static IcebergTableProvider provider; + private static final IcebergTableProvider PROVIDER = IcebergTableProvider.create(); private static final BigqueryClient BQ_CLIENT = new BigqueryClient("IcebergReadWriteIT"); static final String DATASET = "iceberg_sql_tests_" + System.nanoTime(); @@ -102,15 +102,14 @@ public static void createDataset() throws IOException, InterruptedException { TestPipeline.testingPipelineOptions().getTempLocation(), IcebergReadWriteIT.class.getSimpleName(), UUID.randomUUID()); - provider = - IcebergTableProvider.create() - .withCatalogProperties( - ImmutableMap.of( - "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", - "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", - "warehouse", warehouse, - "gcp_project", OPTIONS.getProject(), - "gcp_region", "us-central1")); + PROVIDER.initialize( + IcebergReadWriteIT.class.getSimpleName(), + ImmutableMap.of( + "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", + "warehouse", warehouse, + "gcp_project", OPTIONS.getProject(), + "gcp_region", "us-central1")); BQ_CLIENT.createNewDataset(OPTIONS.getProject(), DATASET); } @@ -121,7 +120,7 @@ public static void deleteDataset() { @Test public void testSqlWriteAndRead() throws IOException, InterruptedException { - BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(provider); + BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(PROVIDER); String tableIdentifier = DATASET + "." + testName.getMethodName(); // 1) create beam table @@ -199,7 +198,7 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { @Test public void testSQLReadWithProjectAndFilterPushDown() { - BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(provider); + BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(PROVIDER); String tableIdentifier = DATASET + "." + testName.getMethodName(); String createTableStatement = diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java index 550d4264a4bc..7358deb20201 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java @@ -35,17 +35,14 @@ /** UnitTest for {@link IcebergTableProvider}. */ public class IcebergTableProviderTest { private final IcebergTableProvider provider = - IcebergTableProvider.create() - .withCatalogProperties( - ImmutableMap.of( - "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", - "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", - "warehouse", "gs://bucket/warehouse")) - .withHadoopConfProperties( - ImmutableMap.of( - "fs.gs.project.id", "apache-beam-testing", - "foo", "bar")) - .withCatalogName("my_catalog"); + IcebergTableProvider.create( + "test_catalog", + ImmutableMap.of( + "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", + "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", + "warehouse", "gs://bucket/warehouse", + "beam.catalog.test_catalog.hadoop.fs.gs.project.id", "apache-beam-testing", + "beam.catalog.test_catalog.hadoop.foo", "bar")); @Test public void testGetTableType() { From 0158cacaf29ed06583862206a588b2c33452be20 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 11 Jun 2025 17:58:39 -0400 Subject: [PATCH 17/20] sync with HEAD and use new Catalog implementation --- .../IO_Iceberg_Integration_Tests.json | 2 +- ...eam_PostCommit_Python_Xlang_IO_Direct.json | 2 +- .../trigger_files/beam_PostCommit_SQL.json | 2 +- CHANGES.md | 1 + sdks/java/extensions/sql/build.gradle | 1 + .../beam/sdk/extensions/sql/SqlTransform.java | 6 -- .../sql/meta/catalog/InMemoryCatalog.java | 7 +- .../catalog/InMemoryCatalogRegistrar.java | 6 +- .../meta/provider/iceberg/IcebergCatalog.java | 33 +++++++++ .../iceberg/IcebergTableProvider.java | 14 +--- .../sdk/extensions/sql/PubsubToIcebergIT.java | 53 +++++++++----- .../provider/iceberg/IcebergReadWriteIT.java | 70 ++++++++++++++----- .../iceberg/IcebergTableProviderTest.java | 2 +- sdks/java/io/expansion-service/build.gradle | 7 +- sdks/java/io/iceberg/build.gradle | 4 +- 15 files changed, 138 insertions(+), 72 deletions(-) create mode 100644 sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 5d04b2c0a8c7..89e73b29da05 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 5 + "modification": 6 } diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json index e0266d62f2e0..f1ba03a243ee 100644 --- a/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Python_Xlang_IO_Direct.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 4 + "modification": 5 } diff --git a/.github/trigger_files/beam_PostCommit_SQL.json b/.github/trigger_files/beam_PostCommit_SQL.json index e3d6056a5de9..b26833333238 100644 --- a/.github/trigger_files/beam_PostCommit_SQL.json +++ b/.github/trigger_files/beam_PostCommit_SQL.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 2 } diff --git a/CHANGES.md b/CHANGES.md index 1089c4f77a49..aa02951ef826 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -80,6 +80,7 @@ * [IcebergIO] Support reading with pushdown filtering ([#34827](https://github.com/apache/beam/pull/34827)) ## New Features / Improvements +* [Beam SQL] Introducing Beam Catalogs ([#35223](https://github.com/apache/beam/pull/35223)) * Adding Google Storage Requests Pays feature (Golang)([#30747](https://github.com/apache/beam/issues/30747)). * X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * [Python] Prism runner now auto-enabled for some Python pipelines using the direct runner ([#34921](https://github.com/apache/beam/pull/34921)). diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle index 2b6197401e1b..a73bd2518fe0 100644 --- a/sdks/java/extensions/sql/build.gradle +++ b/sdks/java/extensions/sql/build.gradle @@ -77,6 +77,7 @@ dependencies { implementation project(":sdks:java:managed") implementation project(":sdks:java:io:iceberg") runtimeOnly project(":sdks:java:io:iceberg:bqms") + runtimeOnly project(":sdks:java:io:iceberg:hive") implementation project(":sdks:java:extensions:avro") implementation project(":sdks:java:extensions:join-library") permitUnusedDeclared project(":sdks:java:extensions:join-library") // BEAM-11761 diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java index 91cf9769fa51..f9cc1fd9d482 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java @@ -149,12 +149,6 @@ public PCollection expand(PInput input) { sqlEnvBuilder.autoLoadUserDefinedFunctions(); ServiceLoader.load(TableProvider.class).forEach(catalogManager::registerTableProvider); } - // register user-specified providers - for (Map.Entry entry : tableProviderMap().entrySet()) { - if (!metaTableProvider.hasProvider(entry.getValue())) { - metaTableProvider.registerProvider(entry.getValue()); - } - } tableProviderMap().forEach(sqlEnvBuilder::addSchema); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java index 4ff9811605a1..8757c2357861 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalog.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.extensions.sql.meta.catalog; -import com.google.auto.service.AutoService; import java.util.Map; import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; import org.apache.beam.sdk.util.Preconditions; -@AutoService(Catalog.class) public class InMemoryCatalog implements Catalog { private final String name; private final Map properties; - private final InMemoryMetaStore metaStore = new InMemoryMetaStore(); + protected final InMemoryMetaStore metaStore = new InMemoryMetaStore(); public InMemoryCatalog(String name, Map properties) { this.name = name; @@ -41,7 +39,8 @@ public String type() { @Override public String name() { - return Preconditions.checkStateNotNull(name, "InMemoryCatalog has not been initialized"); + return Preconditions.checkStateNotNull( + name, getClass().getSimpleName() + " has not been initialized"); } @Override diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogRegistrar.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogRegistrar.java index df53818823ca..2d94e19c1689 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogRegistrar.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/InMemoryCatalogRegistrar.java @@ -18,12 +18,16 @@ package org.apache.beam.sdk.extensions.sql.meta.catalog; import com.google.auto.service.AutoService; +import org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergCatalog; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @AutoService(CatalogRegistrar.class) public class InMemoryCatalogRegistrar implements CatalogRegistrar { @Override public Iterable> getCatalogs() { - return ImmutableList.of(InMemoryCatalog.class); + return ImmutableList.>builder() + .add(InMemoryCatalog.class) + .add(IcebergCatalog.class) + .build(); } } diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java new file mode 100644 index 000000000000..85fd6f4efc17 --- /dev/null +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergCatalog.java @@ -0,0 +1,33 @@ +/* + * 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.beam.sdk.extensions.sql.meta.provider.iceberg; + +import java.util.Map; +import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalog; + +public class IcebergCatalog extends InMemoryCatalog { + public IcebergCatalog(String name, Map properties) { + super(name, properties); + metaStore.registerProvider(new IcebergTableProvider(name, properties)); + } + + @Override + public String type() { + return "iceberg"; + } +} diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java index 49395922af7b..38acc65d6d76 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProvider.java @@ -27,19 +27,9 @@ public class IcebergTableProvider extends InMemoryMetaTableProvider { private static final String BEAM_HADOOP_PREFIX = "beam.catalog.%s.hadoop"; - @VisibleForTesting IcebergCatalogConfig catalogConfig = IcebergCatalogConfig.builder().build(); + @VisibleForTesting final IcebergCatalogConfig catalogConfig; - public static IcebergTableProvider create() { - return new IcebergTableProvider(); - } - - static IcebergTableProvider create(String name, Map properties) { - IcebergTableProvider provider = new IcebergTableProvider(); - provider.initialize(name, properties); - return provider; - } - - public void initialize(String name, Map properties) { + public IcebergTableProvider(String name, Map properties) { ImmutableMap.Builder catalogProps = ImmutableMap.builder(); ImmutableMap.Builder hadoopProps = ImmutableMap.builder(); String hadoopPrefix = String.format(BEAM_HADOOP_PREFIX, name); diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java index 76e005c3d61a..e4800a8ac2a2 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToIcebergIT.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql; +import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.beam.sdk.schemas.Schema.FieldType.INT64; import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING; @@ -25,11 +26,9 @@ import java.io.IOException; import java.io.Serializable; import java.util.List; -import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; -import org.apache.beam.sdk.extensions.sql.meta.provider.iceberg.IcebergTableProvider; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.io.gcp.pubsub.TestPubsub; @@ -62,7 +61,6 @@ public class PubsubToIcebergIT implements Serializable { @Rule public transient TestPipeline pipeline = TestPipeline.create(); @Rule public transient TestPubsub pubsub = TestPubsub.create(); - private static final IcebergTableProvider PROVIDER = IcebergTableProvider.create(); private static final BigqueryClient BQ_CLIENT = new BigqueryClient("PubsubToIcebergIT"); static final String DATASET = "sql_pubsub_to_iceberg_it_" + System.nanoTime(); static String warehouse; @@ -78,19 +76,22 @@ public static void createDataset() throws IOException, InterruptedException { TestPipeline.testingPipelineOptions().getTempLocation(), PubsubToIcebergIT.class.getSimpleName(), UUID.randomUUID()); - icebergConfig = - ImmutableMap.of( - "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", - "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", - "warehouse", warehouse, - "gcp_project", OPTIONS.getProject(), - "gcp_region", "us-central1"); - PROVIDER.initialize(PubsubToIcebergIT.class.getSimpleName(), icebergConfig); BQ_CLIENT.createNewDataset(OPTIONS.getProject(), DATASET); + createCatalogDdl = + "CREATE CATALOG my_catalog \n" + + "TYPE iceberg \n" + + "PROPERTIES (\n" + + " 'catalog-impl' = 'org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog', \n" + + " 'io-impl' = 'org.apache.iceberg.gcp.gcs.GCSFileIO', \n" + + format(" 'warehouse' = '%s', \n", warehouse) + + format(" 'gcp_project' = '%s', \n", OPTIONS.getProject()) + + " 'gcp_region' = 'us-central1')"; + setCatalogDdl = "SET CATALOG my_catalog"; } private String tableIdentifier; - private static Map icebergConfig; + private static String createCatalogDdl; + private static String setCatalogDdl; @Before public void setup() { @@ -136,9 +137,10 @@ public void testSimpleInsert() throws Exception { + "FROM pubsub_topic"; pipeline.apply( SqlTransform.query(insertStatement) + .withDdlString(createCatalogDdl) + .withDdlString(setCatalogDdl) .withDdlString(pubsubTableString) - .withDdlString(icebergTableString) - .withTableProvider("iceberg", PROVIDER)); + .withDdlString(icebergTableString)); pipeline.run(); // Block until a subscription for this topic exists @@ -185,11 +187,21 @@ public void testSimpleInsertFlat() throws Exception { pipeline.apply( SqlTransform.query(insertStatement) + .withDdlString(createCatalogDdl) + .withDdlString(setCatalogDdl) .withDdlString(pubsubTableString) - .withDdlString(bqTableString) - .withTableProvider("iceberg", PROVIDER)); + .withDdlString(bqTableString)); pipeline.run(); + // Block until a subscription for this topic exists + pubsub.assertSubscriptionEventuallyCreated( + pipeline.getOptions().as(GcpOptions.class).getProject(), Duration.standardMinutes(5)); + + List messages = + ImmutableList.of( + message(ts(1), 3, "foo"), message(ts(2), 5, "bar"), message(ts(3), 7, "baz")); + pubsub.publish(messages); + validateRowsWritten(); } @@ -209,19 +221,22 @@ private void validateRowsWritten() throws IOException, InterruptedException { .backoff(); Sleeper sleeper = Sleeper.DEFAULT; do { - List returnedRows = ImmutableList.of(); + List returnedRows; try { returnedRows = BQ_CLIENT.queryUnflattened(query, OPTIONS.getProject(), true, true); - } catch (Exception ignored) { + } catch (Exception e) { + throw new RuntimeException(e); } List beamRows = returnedRows.stream() .map(r -> BigQueryUtils.toBeamRow(SOURCE_SCHEMA, r)) .collect(Collectors.toList()); if (beamRows.containsAll(expectedRows)) { - break; + return; } } while (BackOffUtils.next(sleeper, backOff)); + + throw new RuntimeException("Polled for 5 minutes and could not find all rows in table."); } private Row row(Schema schema, Object... values) { diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java index 9a5749c26032..fd3c18b6072a 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergReadWriteIT.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.extensions.sql.meta.provider.iceberg; +import static java.lang.String.format; import static java.util.Arrays.asList; import static org.apache.beam.sdk.extensions.sql.utils.DateTimeUtils.parseTimestampWithUTCTimeZone; import static org.apache.beam.sdk.schemas.Schema.FieldType.BOOLEAN; @@ -44,14 +45,15 @@ import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode; import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils; import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils; +import org.apache.beam.sdk.extensions.sql.meta.catalog.InMemoryCatalogManager; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils; import org.apache.beam.sdk.io.gcp.testing.BigqueryClient; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Duration; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -86,7 +88,6 @@ public class IcebergReadWriteIT { @Rule public transient TestPipeline writePipeline = TestPipeline.create(); @Rule public transient TestPipeline readPipeline = TestPipeline.create(); @Rule public TestName testName = new TestName(); - private static final IcebergTableProvider PROVIDER = IcebergTableProvider.create(); private static final BigqueryClient BQ_CLIENT = new BigqueryClient("IcebergReadWriteIT"); static final String DATASET = "iceberg_sql_tests_" + System.nanoTime(); @@ -97,19 +98,11 @@ public class IcebergReadWriteIT { @BeforeClass public static void createDataset() throws IOException, InterruptedException { warehouse = - String.format( + format( "%s%s/%s", TestPipeline.testingPipelineOptions().getTempLocation(), IcebergReadWriteIT.class.getSimpleName(), UUID.randomUUID()); - PROVIDER.initialize( - IcebergReadWriteIT.class.getSimpleName(), - ImmutableMap.of( - "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", - "io-impl", "org.apache.iceberg.gcp.gcs.GCSFileIO", - "warehouse", warehouse, - "gcp_project", OPTIONS.getProject(), - "gcp_region", "us-central1")); BQ_CLIENT.createNewDataset(OPTIONS.getProject(), DATASET); } @@ -120,10 +113,29 @@ public static void deleteDataset() { @Test public void testSqlWriteAndRead() throws IOException, InterruptedException { - BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(PROVIDER); + BeamSqlEnv sqlEnv = + BeamSqlEnv.builder(new InMemoryCatalogManager()) + .setPipelineOptions(PipelineOptionsFactory.create()) + .build(); String tableIdentifier = DATASET + "." + testName.getMethodName(); - // 1) create beam table + // 1) create Iceberg catalog + String createCatalog = + "CREATE CATALOG my_catalog \n" + + "TYPE iceberg \n" + + "PROPERTIES (\n" + + " 'catalog-impl' = 'org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog', \n" + + " 'io-impl' = 'org.apache.iceberg.gcp.gcs.GCSFileIO', \n" + + format(" 'warehouse' = '%s', \n", warehouse) + + format(" 'gcp_project' = '%s', \n", OPTIONS.getProject()) + + " 'gcp_region' = 'us-central1')"; + sqlEnv.executeDdl(createCatalog); + + // 2) use the catalog we just created + String setCatalog = "SET CATALOG my_catalog"; + sqlEnv.executeDdl(setCatalog); + + // 3) create beam table String createTableStatement = "CREATE EXTERNAL TABLE TEST( \n" + " c_bigint BIGINT, \n" @@ -143,7 +155,7 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { + "'"; sqlEnv.executeDdl(createTableStatement); - // 2) write to underlying Iceberg table + // 3) write to underlying Iceberg table String insertStatement = "INSERT INTO TEST VALUES (" + "9223372036854775807, " @@ -163,8 +175,8 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { BeamSqlRelUtils.toPCollection(writePipeline, sqlEnv.parseQuery(insertStatement)); writePipeline.run().waitUntilFinish(); - // 3) run external query on Iceberg table (hosted on BQ) to verify correct row was written - String query = String.format("SELECT * FROM `%s.%s`", OPTIONS.getProject(), tableIdentifier); + // 4) run external query on Iceberg table (hosted on BQ) to verify correct row was written + String query = format("SELECT * FROM `%s.%s`", OPTIONS.getProject(), tableIdentifier); TableRow returnedRow = BQ_CLIENT.queryUnflattened(query, OPTIONS.getProject(), true, true).get(0); Row beamRow = BigQueryUtils.toBeamRow(SOURCE_SCHEMA, returnedRow); @@ -187,7 +199,7 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { .build(); assertEquals(expectedRow, beamRow); - // 4) read using Beam SQL and verify + // 5) read using Beam SQL and verify String selectTableStatement = "SELECT * FROM TEST"; PCollection output = BeamSqlRelUtils.toPCollection(readPipeline, sqlEnv.parseQuery(selectTableStatement)); @@ -198,9 +210,29 @@ public void testSqlWriteAndRead() throws IOException, InterruptedException { @Test public void testSQLReadWithProjectAndFilterPushDown() { - BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(PROVIDER); + BeamSqlEnv sqlEnv = + BeamSqlEnv.builder(new InMemoryCatalogManager()) + .setPipelineOptions(PipelineOptionsFactory.create()) + .build(); String tableIdentifier = DATASET + "." + testName.getMethodName(); + // 1) create Iceberg catalog + String createCatalog = + "CREATE CATALOG my_catalog \n" + + "TYPE iceberg \n" + + "PROPERTIES (\n" + + " 'catalog-impl' = 'org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog', \n" + + " 'io-impl' = 'org.apache.iceberg.gcp.gcs.GCSFileIO', \n" + + format(" 'warehouse' = '%s', \n", warehouse) + + format(" 'gcp_project' = '%s', \n", OPTIONS.getProject()) + + " 'gcp_region' = 'us-central1')"; + sqlEnv.executeDdl(createCatalog); + + // 2) use the catalog we just created + String setCatalog = "SET CATALOG my_catalog"; + sqlEnv.executeDdl(setCatalog); + + // 3) create Beam table String createTableStatement = "CREATE EXTERNAL TABLE TEST( \n" + " c_integer INTEGER, \n" @@ -215,6 +247,7 @@ public void testSQLReadWithProjectAndFilterPushDown() { + "'"; sqlEnv.executeDdl(createTableStatement); + // 4) insert some data) String insertStatement = "INSERT INTO TEST VALUES " + "(123, 1.23, TRUE, TIMESTAMP '2025-05-22 20:17:40.123', 'a'), " @@ -223,6 +256,7 @@ public void testSQLReadWithProjectAndFilterPushDown() { BeamSqlRelUtils.toPCollection(writePipeline, sqlEnv.parseQuery(insertStatement)); writePipeline.run().waitUntilFinish(Duration.standardMinutes(5)); + // 5) read with a filter String selectTableStatement = "SELECT c_integer, c_varchar FROM TEST where " + "(c_boolean=TRUE and c_varchar in ('a', 'b')) or c_float > 5"; diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java index 7358deb20201..316028d7599f 100644 --- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java +++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/iceberg/IcebergTableProviderTest.java @@ -35,7 +35,7 @@ /** UnitTest for {@link IcebergTableProvider}. */ public class IcebergTableProviderTest { private final IcebergTableProvider provider = - IcebergTableProvider.create( + new IcebergTableProvider( "test_catalog", ImmutableMap.of( "catalog-impl", "org.apache.iceberg.gcp.bigquery.BigQueryMetastoreCatalog", diff --git a/sdks/java/io/expansion-service/build.gradle b/sdks/java/io/expansion-service/build.gradle index 6655df7d80c6..2e6274571a89 100644 --- a/sdks/java/io/expansion-service/build.gradle +++ b/sdks/java/io/expansion-service/build.gradle @@ -71,13 +71,8 @@ dependencies { implementation project(":sdks:java:io:kafka:upgrade") permitUnusedDeclared project(":sdks:java:io:kafka:upgrade") // BEAM-11761 - // **** IcebergIO runtime dependencies **** - runtimeOnly library.java.hadoop_auth - runtimeOnly library.java.hadoop_client - // For writing to GCS - runtimeOnly library.java.bigdataoss_gcs_connector + // **** IcebergIO catalogs **** // HiveCatalog - runtimeOnly ("org.apache.iceberg:iceberg-hive-metastore:1.4.2") runtimeOnly project(path: ":sdks:java:io:iceberg:hive") // BigQueryMetastoreCatalog (Java 11+) runtimeOnly project(path: ":sdks:java:io:iceberg:bqms", configuration: "shadow") diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 4d79fb061d05..2ac04eb067a3 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -63,11 +63,11 @@ dependencies { permitUnusedDeclared "org.immutables:value:2.8.8" implementation library.java.vendored_calcite_1_28_0 runtimeOnly "org.apache.iceberg:iceberg-gcp:$iceberg_version" + runtimeOnly library.java.bigdataoss_gcs_connector + runtimeOnly library.java.hadoop_client testImplementation project(":sdks:java:managed") - testImplementation library.java.hadoop_client testImplementation library.java.bigdataoss_gcsio - testImplementation library.java.bigdataoss_gcs_connector testImplementation library.java.bigdataoss_util_hadoop testImplementation "org.apache.parquet:parquet-avro:$parquet_version" testImplementation "org.apache.parquet:parquet-common:$parquet_version" From f5eb78d378e18f01a3d16c81c5bd82084855fa76 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 11 Jun 2025 20:04:53 -0400 Subject: [PATCH 18/20] mark new interfaces @internal --- .../apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java | 3 +++ .../beam/sdk/extensions/sql/meta/catalog/CatalogManager.java | 3 +++ .../beam/sdk/extensions/sql/meta/catalog/CatalogRegistrar.java | 3 +++ 3 files changed, 9 insertions(+) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java index 08ca3488c1cc..41664bcf2eb6 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.extensions.sql.meta.catalog; import java.util.Map; + +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; /** @@ -25,6 +27,7 @@ * configuration properties. Uses an underlying {@link MetaStore} to manage tables and table * providers. */ +@Internal public interface Catalog { /** A type that defines this catalog. */ String type(); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java index 2ac5266219bc..53ed68666117 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java @@ -18,6 +18,8 @@ package org.apache.beam.sdk.extensions.sql.meta.catalog; import java.util.Map; + +import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; import org.checkerframework.checker.nullness.qual.Nullable; @@ -32,6 +34,7 @@ *

When {@link #registerTableProvider(String, TableProvider)} is called, the provider should * become available for all catalogs. */ +@Internal public interface CatalogManager { /** Creates and stores a catalog of a particular type. */ void createCatalog(String name, String type, Map properties); diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogRegistrar.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogRegistrar.java index 3fad539e8cc4..08fe6179b1f1 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogRegistrar.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogRegistrar.java @@ -17,11 +17,14 @@ */ package org.apache.beam.sdk.extensions.sql.meta.catalog; +import org.apache.beam.sdk.annotations.Internal; + /** * Over-arching registrar to capture available {@link Catalog}s. Implementations should be marked * with {@link com.google.auto.service.AutoService} to be available to {@link * java.util.ServiceLoader}s. */ +@Internal public interface CatalogRegistrar { Iterable> getCatalogs(); } From 73b6087f8287a0cd2c36da38c853e80554bb8299 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 11 Jun 2025 20:16:37 -0400 Subject: [PATCH 19/20] spotless --- .../org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java | 1 - .../beam/sdk/extensions/sql/meta/catalog/CatalogManager.java | 1 - 2 files changed, 2 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java index 41664bcf2eb6..2a99209e06f5 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/Catalog.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.extensions.sql.meta.catalog; import java.util.Map; - import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore; diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java index 53ed68666117..4654f0dd1b0d 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/catalog/CatalogManager.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.extensions.sql.meta.catalog; import java.util.Map; - import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema; import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider; From 5dd2a120082b7508902fb5e7758d38a6d3752dee Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 12 Jun 2025 11:18:10 -0400 Subject: [PATCH 20/20] fix unparse method --- .../sql/impl/parser/SqlCreateCatalog.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java index 7265c3103c16..c1d96eea7bae 100644 --- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java +++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateCatalog.java @@ -99,16 +99,16 @@ public void unparse(SqlWriter writer, int leftPrec, int rightPrec) { if (i > 0) { writer.keyword(","); } - properties.get(i).unparse(writer, leftPrec, rightPrec); - } - - for (int i = 0; i < properties.size(); i += 2) { - if (i > 0) { - writer.keyword(","); - } - properties.get(i).unparse(writer, leftPrec, rightPrec); // key + SqlNode property = properties.get(i); + checkState( + property instanceof SqlNodeList, + String.format( + "Unexpected properties entry '%s' of class '%s'", property, property.getClass())); + SqlNodeList kv = ((SqlNodeList) property); + + kv.get(0).unparse(writer, leftPrec, rightPrec); // key writer.keyword("="); - properties.get(i + 1).unparse(writer, leftPrec, rightPrec); // value + kv.get(1).unparse(writer, leftPrec, rightPrec); // value } writer.keyword(")"); }