diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 7ab7bcd9a9c6..37dd25bf9029 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": 2 + "modification": 3 } diff --git a/CHANGES.md b/CHANGES.md index c75a42d25945..fcc590a72135 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -65,6 +65,7 @@ * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * [Java] Use API compatible with both com.google.cloud.bigdataoss:util 2.x and 3.x in BatchLoads ([#34105](https://github.com/apache/beam/pull/34105)) +* [Iceberg] Added new CDC source for batch and streaming, available as `Managed.ICEBERG_CDC` ([#33504](https://github.com/apache/beam/pull/33504)) ## New Features / Improvements diff --git a/contributor-docs/discussion-docs/2025.md b/contributor-docs/discussion-docs/2025.md index 30ba04d3ef06..b3969b1e2b06 100644 --- a/contributor-docs/discussion-docs/2025.md +++ b/contributor-docs/discussion-docs/2025.md @@ -15,5 +15,9 @@ limitations under the License. # List Of Documents Submitted To dev@beam.apache.org In 2025 | No. | Author | Subject | Date (UTC) | |---|---|---|---| -| 1 | Danny McCormick | [Beam Python Dependency Extras](https://docs.google.com/document/d/1c84Gc-cZRCfrU8f7kWGsNR2o8oSRjCM-dGHO9KvPWPw) | 2025-01-27 17:50:00 | -| 2 | Danny McCormick | [How vLLM Model Handler Works (Plus a Summary of Model Memory Management in Beam ML)](https://docs.google.com/document/d/1UB4umrtnp1Eg45fiUB3iLS7kPK3BE6pcf0YRDkA289Q) | 2025-01-31 17:50:00 | +| 1 | Kenneth Knowles | [Apache Beam Release Acceptance Criteria - Google Sheets](https://docs.google.com/spreadsheets/d/1qk-N5vjXvbcEk68GjbkSZTR8AGqyNUM-oLFo_ZXBpJw) | 2025-01-13 10:54:22 | +| 2 | Danny McCormick | [Apache Beam Vendored Dependencies Release Guide](https://s.apache.org/beam-release-vendored-artifacts) | 2025-01-13 15:00:51 | +| 3 | Danny McCormick | [Beam Python & ML Dependency Extras](https://docs.google.com/document/d/1c84Gc-cZRCfrU8f7kWGsNR2o8oSRjCM-dGHO9KvPWPw) | 2025-01-27 15:33:36 | +| 4 | Danny McCormick | [How vLLM Model Handler Works (Plus a Summary of Model Memory Management in Beam ML)](https://docs.google.com/document/d/1UB4umrtnp1Eg45fiUB3iLS7kPK3BE6pcf0YRDkA289Q) | 2025-01-31 11:56:59 | +| 5 | Shunping Huang | [Improve Logging Dependencies in Beam Java SDK](https://docs.google.com/document/d/1IkbiM4m8D-aB3NYI1aErFZHt6M7BQ-8eCULh284Davs) | 2025-02-04 15:13:14 | +| 6 | Ahmed Abualsaud | [Iceberg Incremental Source design](https://s.apache.org/beam-iceberg-incremental-source) | 2025-03-03 14:52:42 | \ No newline at end of file diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto index f102e82bafa6..add8a1999caf 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/external_transforms.proto @@ -74,6 +74,8 @@ message ManagedTransforms { "beam:schematransform:org.apache.beam:bigquery_storage_read:v1"]; BIGQUERY_WRITE = 5 [(org.apache.beam.model.pipeline.v1.beam_urn) = "beam:schematransform:org.apache.beam:bigquery_write:v1"]; + ICEBERG_CDC_READ = 6 [(org.apache.beam.model.pipeline.v1.beam_urn) = + "beam:schematransform:org.apache.beam:iceberg_cdc_read:v1"]; } } diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index be1541d6e68c..32c95ea8260c 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -50,6 +50,7 @@ dependencies { implementation library.java.slf4j_api implementation library.java.joda_time implementation "org.apache.parquet:parquet-column:$parquet_version" + implementation "org.apache.parquet:parquet-hadoop:$parquet_version" implementation "org.apache.orc:orc-core:$orc_version" implementation "org.apache.iceberg:iceberg-core:$iceberg_version" implementation "org.apache.iceberg:iceberg-api:$iceberg_version" @@ -88,7 +89,7 @@ dependencies { testImplementation library.java.google_api_services_bigquery testRuntimeOnly library.java.slf4j_jdk14 - testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":runners:direct-java", configuration: "shadow") testRuntimeOnly project(path: ":runners:google-cloud-dataflow-java") hadoopVersions.each {kv -> "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-client:$kv.value" diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java index fed72a381d5e..18117dd9ed88 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/AppendFilesToTables.java @@ -82,7 +82,7 @@ public String apply(FileWriteResult input) { .apply( "Append metadata updates to tables", ParDo.of(new AppendFilesToTablesDoFn(catalogConfig, manifestFilePrefix))) - .setCoder(KvCoder.of(StringUtf8Coder.of(), SnapshotInfo.CODER)); + .setCoder(KvCoder.of(StringUtf8Coder.of(), SnapshotInfo.getCoder())); } private static class AppendFilesToTablesDoFn diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/CreateReadTasksDoFn.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/CreateReadTasksDoFn.java new file mode 100644 index 000000000000..751610c599ef --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/CreateReadTasksDoFn.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.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ExecutionException; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.IncrementalAppendScan; +import org.apache.iceberg.ScanTaskParser; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Scans the given snapshot and creates multiple {@link ReadTask}s. Each task represents a portion + * of a data file that was appended within the snapshot range. + */ +class CreateReadTasksDoFn + extends DoFn>, KV> { + private static final Logger LOG = LoggerFactory.getLogger(CreateReadTasksDoFn.class); + private static final Counter totalScanTasks = + Metrics.counter(CreateReadTasksDoFn.class, "totalScanTasks"); + // TODO(ahmedabu98): should we expose a metric that tracks the latest observed snapshot sequence + // number? + + private final IcebergScanConfig scanConfig; + + CreateReadTasksDoFn(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + } + + @ProcessElement + public void process( + @Element KV> element, + OutputReceiver> out) + throws IOException, ExecutionException { + Table table = + TableCache.getRefreshed(element.getKey(), scanConfig.getCatalogConfig().catalog()); + List snapshots = element.getValue(); + + // scan snapshots individually and assign commit timestamp to files + for (SnapshotInfo snapshot : snapshots) { + @Nullable Long fromSnapshot = snapshot.getParentId(); + long toSnapshot = snapshot.getSnapshotId(); + + if (!DataOperations.APPEND.equals(snapshot.getOperation())) { + LOG.info( + "Skipping non-append snapshot of operation '{}'. Sequence number: {}, id: {}", + snapshot.getOperation(), + snapshot.getSequenceNumber(), + snapshot.getSnapshotId()); + } + + LOG.info("Planning to scan snapshot {}", toSnapshot); + IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); + if (fromSnapshot != null) { + scan = scan.fromSnapshotExclusive(fromSnapshot); + } + + createAndOutputReadTasks(scan, snapshot, out); + } + } + + private void createAndOutputReadTasks( + IncrementalAppendScan scan, + SnapshotInfo snapshot, + OutputReceiver> out) + throws IOException { + int numTasks = 0; + try (CloseableIterable combinedScanTasks = scan.planTasks()) { + for (CombinedScanTask combinedScanTask : combinedScanTasks) { + // A single DataFile can be broken up into multiple FileScanTasks + for (FileScanTask fileScanTask : combinedScanTask.tasks()) { + ReadTask task = + ReadTask.builder() + .setFileScanTaskJson(ScanTaskParser.toJson(fileScanTask)) + .setByteSize(fileScanTask.file().fileSizeInBytes()) + .setOperation(snapshot.getOperation()) + .setSnapshotTimestampMillis(snapshot.getTimestampMillis()) + .build(); + ReadTaskDescriptor descriptor = + ReadTaskDescriptor.builder() + .setTableIdentifierString(checkStateNotNull(snapshot.getTableIdentifierString())) + .build(); + + out.outputWithTimestamp( + KV.of(descriptor, task), Instant.ofEpochMilli(snapshot.getTimestampMillis())); + totalScanTasks.inc(); + numTasks++; + } + } + } + LOG.info("Snapshot {} produced {} read tasks.", snapshot.getSnapshotId(), numTasks); + } +} 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 1d89a6a26e98..c3a185f3a833 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 @@ -51,26 +51,25 @@ public static Builder builder() { } public org.apache.iceberg.catalog.Catalog catalog() { - if (cachedCatalog != null) { - return cachedCatalog; + if (cachedCatalog == null) { + String catalogName = getCatalogName(); + if (catalogName == null) { + catalogName = "apache-beam-" + ReleaseInfo.getReleaseInfo().getVersion(); + } + Map catalogProps = getCatalogProperties(); + if (catalogProps == null) { + catalogProps = Maps.newHashMap(); + } + Map confProps = getConfigProperties(); + if (confProps == null) { + confProps = Maps.newHashMap(); + } + Configuration config = new Configuration(); + for (Map.Entry prop : confProps.entrySet()) { + config.set(prop.getKey(), prop.getValue()); + } + cachedCatalog = CatalogUtil.buildIcebergCatalog(catalogName, catalogProps, config); } - String catalogName = getCatalogName(); - if (catalogName == null) { - catalogName = "apache-beam-" + ReleaseInfo.getReleaseInfo().getVersion(); - } - Map catalogProps = getCatalogProperties(); - if (catalogProps == null) { - catalogProps = Maps.newHashMap(); - } - Map confProps = getConfigProperties(); - if (confProps == null) { - confProps = Maps.newHashMap(); - } - Configuration config = new Configuration(); - for (Map.Entry prop : confProps.entrySet()) { - config.set(prop.getKey(), prop.getValue()); - } - cachedCatalog = CatalogUtil.buildIcebergCatalog(catalogName, catalogProps, config); return cachedCatalog; } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java new file mode 100644 index 000000000000..0064b49475d0 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java @@ -0,0 +1,215 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.io.iceberg.IcebergCdcReadSchemaTransformProvider.Configuration; +import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.beam.model.pipeline.v1.ExternalTransforms; +import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Enums; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; + +/** + * SchemaTransform implementation for {@link IcebergIO#readRows}. Reads records from Iceberg and + * outputs a {@link org.apache.beam.sdk.values.PCollection} of Beam {@link + * org.apache.beam.sdk.values.Row}s. + */ +@AutoService(SchemaTransformProvider.class) +public class IcebergCdcReadSchemaTransformProvider + extends TypedSchemaTransformProvider { + static final String OUTPUT_TAG = "output"; + + @Override + protected SchemaTransform from(Configuration configuration) { + return new IcebergCdcReadSchemaTransform(configuration); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(OUTPUT_TAG); + } + + @Override + public String identifier() { + return getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_CDC_READ); + } + + static class IcebergCdcReadSchemaTransform extends SchemaTransform { + private final Configuration configuration; + + IcebergCdcReadSchemaTransform(Configuration configuration) { + this.configuration = configuration; + } + + Row getConfigurationRow() { + try { + // To stay consistent with our SchemaTransform configuration naming conventions, + // we sort lexicographically and convert field names to snake_case + return SchemaRegistry.createDefault() + .getToRowFunction(Configuration.class) + .apply(configuration) + .sorted() + .toSnakeCase(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + @Nullable String strategyStr = configuration.getStartingStrategy(); + StartingStrategy strategy = null; + if (strategyStr != null) { + Optional optional = + Enums.getIfPresent(StartingStrategy.class, strategyStr.toUpperCase()); + if (!optional.isPresent()) { + throw new IllegalArgumentException( + "Invalid starting strategy. Valid values are: " + + Arrays.toString(StartingStrategy.values())); + } + strategy = optional.get(); + } + + IcebergIO.ReadRows readRows = + IcebergIO.readRows(configuration.getIcebergCatalog()) + .withCdc() + .from(TableIdentifier.parse(configuration.getTable())) + .fromSnapshot(configuration.getFromSnapshot()) + .toSnapshot(configuration.getToSnapshot()) + .fromTimestamp(configuration.getFromTimestamp()) + .toTimestamp(configuration.getToTimestamp()) + .withStartingStrategy(strategy) + .streaming(configuration.getStreaming()); + + @Nullable Integer pollIntervalSeconds = configuration.getPollIntervalSeconds(); + if (pollIntervalSeconds != null) { + readRows = readRows.withPollInterval(Duration.standardSeconds(pollIntervalSeconds)); + } + + PCollection output = input.getPipeline().apply(readRows); + + return PCollectionRowTuple.of(OUTPUT_TAG, output); + } + } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Configuration { + static Builder builder() { + return new AutoValue_IcebergCdcReadSchemaTransformProvider_Configuration.Builder(); + } + + @SchemaFieldDescription("Identifier of the Iceberg table.") + abstract String getTable(); + + @SchemaFieldDescription("Name of the catalog containing the table.") + @Nullable + abstract String getCatalogName(); + + @SchemaFieldDescription("Properties used to set up the Iceberg catalog.") + @Nullable + abstract Map getCatalogProperties(); + + @SchemaFieldDescription("Properties passed to the Hadoop Configuration.") + @Nullable + abstract Map getConfigProperties(); + + @SchemaFieldDescription("Starts reading from this snapshot ID (inclusive).") + abstract @Nullable Long getFromSnapshot(); + + @SchemaFieldDescription("Reads up to this snapshot ID (inclusive).") + abstract @Nullable Long getToSnapshot(); + + @SchemaFieldDescription( + "Starts reading from the first snapshot (inclusive) that was created after this timestamp (in milliseconds).") + abstract @Nullable Long getFromTimestamp(); + + @SchemaFieldDescription( + "Reads up to the latest snapshot (inclusive) created before this timestamp (in milliseconds).") + abstract @Nullable Long getToTimestamp(); + + @SchemaFieldDescription( + "The source's starting strategy. Valid options are: \"earliest\" or \"latest\". Can be overriden " + + "by setting a starting snapshot or timestamp. Defaults to earliest for batch, and latest for streaming.") + abstract @Nullable String getStartingStrategy(); + + @SchemaFieldDescription( + "Enables streaming reads, where source continuously polls for snapshots forever.") + abstract @Nullable Boolean getStreaming(); + + @SchemaFieldDescription( + "The interval at which to poll for new snapshots. Defaults to 60 seconds.") + abstract @Nullable Integer getPollIntervalSeconds(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setTable(String table); + + abstract Builder setCatalogName(String catalogName); + + abstract Builder setCatalogProperties(Map catalogProperties); + + abstract Builder setConfigProperties(Map confProperties); + + abstract Builder setFromSnapshot(Long snapshot); + + abstract Builder setToSnapshot(Long snapshot); + + abstract Builder setFromTimestamp(Long timestamp); + + abstract Builder setToTimestamp(Long timestamp); + + abstract Builder setStartingStrategy(String strategy); + + abstract Builder setPollIntervalSeconds(Integer pollInterval); + + abstract Builder setStreaming(Boolean streaming); + + abstract Configuration build(); + } + + IcebergCatalogConfig getIcebergCatalog() { + return IcebergCatalogConfig.builder() + .setCatalogName(getCatalogName()) + .setCatalogProperties(getCatalogProperties()) + .setConfigProperties(getConfigProperties()) + .build(); + } + } +} 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 1d4b36585237..893f4d3d0679 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 @@ -50,7 +50,6 @@ *
{@code
  * Map config = Map.of(
  *         "table", table,
- *         "triggering_frequency_seconds", 5,
  *         "catalog_name", name,
  *         "catalog_properties", Map.of(
  *                 "warehouse", warehouse_path,
@@ -58,21 +57,32 @@
  *         "config_properties", Map.of(
  *                 "hive.metastore.uris", metastore_uri));
  *
+ *
+ * ====== WRITE ======
  * pipeline
  *     .apply(Create.of(BEAM_ROWS))
  *     .apply(Managed.write(ICEBERG).withConfig(config));
  *
  *
- * // ====== READ ======
+ * ====== READ ======
  * pipeline
  *     .apply(Managed.read(ICEBERG).withConfig(config))
  *     .getSinglePCollection()
  *     .apply(ParDo.of(...));
+ *
+ *
+ * ====== READ CDC ======
+ * pipeline
+ *     .apply(Managed.read(ICEBERG_CDC).withConfig(config))
+ *     .getSinglePCollection()
+ *     .apply(ParDo.of(...));
  * }
* - *

Configuration Options

+ * Look for more detailed examples below. * - * + *

Configuration Options

+ * + *
* * * @@ -81,12 +91,6 @@ * template to use dynamic destinations (see the `Dynamic Destinations` section below for details). * * - * - * - * * * * @@ -103,22 +107,98 @@ * *
Parameter Type Description
{@code triggering_frequency_seconds} {@code int} Required for streaming writes. Roughly every - * {@code triggering_frequency_seconds} duration, the sink will write records to data files and produce a table snapshot. - * Generally, a higher value will produce fewer, larger data files. - *
{@code catalog_name} {@code str} The name of the catalog. Defaults to {@code apache-beam-}.
* - *

Additional configuration options are provided in the `Pre-filtering Options` section below, - * for Iceberg writes. + *

Sink-only Options

* - *

Creating Tables

+ * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
Parameter Type Description
{@code triggering_frequency_seconds} {@code int} Required for streaming writes. Roughly every + * {@code triggering_frequency_seconds} duration, the sink will write records to data files and produce a table snapshot. + * Generally, a higher value will produce fewer, larger data files. + *
{@code drop} {@code list} A list of fields to drop before writing to table(s).
{@code keep} {@code list} A list of fields to keep, dropping the rest before writing to table(s).
{@code only} {@code str} A nested record field that should be the only thing written to table(s).
* - *

If an Iceberg table does not exist at the time of writing, this connector will automatically - * create one with the data's schema. + *

Source-only Options

* - *

Note that this is a best-effort operation that depends on the {@link Catalog} implementation. - * Some implementations may not support creating a table using the Iceberg API. + *

CDC Source options

+ * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
Parameter Type Description
{@code streaming} {@code boolean} + * Enables streaming reads. The source will continuously poll for snapshots forever. + *
{@code poll_interval_seconds} {@code int} + * The interval at which to scan the table for new snapshots. Defaults to 60 seconds. Only applicable for streaming reads. + *
{@code from_snapshot} {@code long} Starts reading from this snapshot ID (inclusive). + *
{@code to_snapshot} {@code long} Reads up to this snapshot ID (inclusive). By default, batch reads will read up to the latest snapshot (inclusive), + * while streaming reads will continue polling for new snapshots forever. + *
{@code from_timestamp} {@code long} Starts reading from the earliest snapshot (inclusive) created after this timestamp (in milliseconds). + *
{@code to_timestamp} {@code long} Reads up to the latest snapshot (inclusive) created before this timestamp (in milliseconds). By default, batch reads will read up to the latest snapshot (inclusive), + * while streaming reads will continue polling for new snapshots forever. + *
{@code starting_strategy} {@code str} + * The source's starting strategy. Valid options are: + *
    + *
  • {@code earliest}: starts reading from the earliest snapshot
  • + *
  • {@code latest}: starts reading from the latest snapshot
  • + *
+ *

Defaults to {@code earliest} for batch, and {@code latest} for streaming. + *

* *

Beam Rows

* *

Being a Managed transform, this IO exclusively writes and reads using Beam {@link Row}s. * Conversion takes place between Beam {@link Row}s and Iceberg {@link Record}s using helper methods - * in {@link IcebergUtils}. Below is a type conversion table mapping Beam and Iceberg types: + * in {@link IcebergUtils}. Below is the mapping between Beam and Iceberg types: * * * @@ -197,6 +277,16 @@ *

For Iceberg reads, the connector will produce Beam {@code SqlTypes.DATETIME} types for * Iceberg's {@code timestamp} and {@code DATETIME} types for {@code timestamptz}. * + *

Writing to Tables

+ * + *

Creating Tables

+ * + *

If an Iceberg table does not exist at the time of writing, this connector will automatically + * create one with the data's schema. + * + *

Note that this is a best-effort operation that depends on the {@link Catalog} implementation. + * Some implementations may not support creating a table using the Iceberg API. + * *

Dynamic Destinations

* *

Managed Iceberg supports writing to dynamic destinations. To do so, please provide an @@ -213,23 +303,8 @@ *

Some use cases may benefit from filtering record fields right before the write operation. For * example, you may want to provide meta-data to guide records to the right destination, but not * necessarily write that meta-data to your table. Some light-weight filtering options are provided - * to accommodate such cases, allowing you to control what actually gets written: - * - *

- * - * - * - * - * - * - * - * - * - * - * - * - *
Parameter Type Description
{@code drop} {@code list} Drops the specified fields.
{@code keep} {@code list} Keeps the specified fields and drops the rest.
{@code only} {@code str} Use this to specify a nested record you intend to write. - * That record wll be written and the rest will be dropped.
+ * to accommodate such cases, allowing you to control what actually gets written (see {@code + * drop}, {@code keep}, {@code only}}). * *

Example write to dynamic destinations (pseudocode): * @@ -268,10 +343,11 @@ * *

When records are written and committed to a table, a snapshot is produced. A batch pipeline * will perform a single commit and create a single snapshot per table. A streaming pipeline will - * produce a snapshot roughly according to the configured {@code triggering_frequency_seconds}. + * produce a snapshot roughly according to the configured {@code + * triggering_frequency_seconds}. * - *

You can access these snapshots and perform downstream processing by fetching the {@code - * "snapshots"} output PCollection: + *

You can access these snapshots and perform downstream processing by fetching the {@code + * "snapshots"} output PCollection: * *

{@code
  * pipeline
@@ -310,7 +386,137 @@
  *   
  * 
  *
- * 

For internal use only; no backwards compatibility guarantees + *
+ *
+ * + *

Reading from Tables

+ * + * With the following configuration, + * + *
{@code
+ * Map config = Map.of(
+ *         "table", table,
+ *         "catalog_name", name,
+ *         "catalog_properties", Map.of(...),
+ *         "config_properties", Map.of(...));
+ * }
+ * + * Example of a simple batch read: + * + *
{@code
+ * PCollection rows = pipeline
+ *     .apply(Managed.read(ICEBERG).withConfig(config))
+ *     .getSinglePCollection();
+ * }
+ * + * Example of a simple CDC read: + * + *
{@code
+ * PCollection output = pipeline
+ *     .apply(Managed.read(ICEBERG_CDC).withConfig(config))
+ *     .getSinglePCollection();
+ *
+ * PCollection rows = output
+ *     .apply(ReadUtils.extractRecords());
+ * }
+ * + *

Note: This reads append-only snapshots. Full CDC is not supported yet. + * + *

The CDC streaming source (enabled with {@code streaming=true}) continuously polls the + * table for new snapshots, with a default interval of 60 seconds. This can be overridden using + * {@code poll_interval_seconds}: + * + *

{@code
+ * config.put("streaming", true);
+ * config.put("poll_interval_seconds", 10);
+ * }
+ * + *

Output Schema

+ * + *

Reading with {@code Managed.read(ICEBERG)} produces a {@code PCollection} + * containing data records that conform to the table schema. + * + *

Reading with {@code Managed.read(ICEBERG_CDC)} produces a {@code + * PCollection} with the following schema: + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
Field Type Description
{@code record} {@code Beam Row} + * The data record. + *
{@code operation} {@code string} + * The snapshot operation associated with this record. For now, only "append" is supported. + *
+ * + *

Choosing a Starting Point (CDC only)

+ * + * By default, a batch read will start reading from the earliest (oldest) table snapshot. A + * streaming read will start reading from the latest (most recent) snapshot. This behavior can be + * overridden in a few mutually exclusive ways: + * + *
    + *
  • Manually setting a starting strategy with {@code starting_strategy} to be {@code + * "earliest"} or {@code "latest"}. + *
  • Setting a starting snapshot id with {@code from_snapshot}. + *
  • Setting a starting timestamp (milliseconds) with {@code from_timestamp}. + *
+ * + *

For example: + * + *

{@code
+ * Map config = Map.of(
+ *         "table", table,
+ *         "catalog_name", name,
+ *         "catalog_properties", Map.of(...),
+ *         "config_properties", Map.of(...),
+ *         "streaming", true,
+ *         "from_snapshot", 123456789L);
+ *
+ * PCollection = pipeline
+ *     .apply(Managed.read(ICEBERG_CDC).withConfig(config))
+ *     .getSinglePCollection();
+ * }
+ * + *

Choosing an End Point (CDC only)

+ * + * By default, a batch read will go up until the most recent table snapshot. A streaming read will + * continue monitoring the table for new snapshots forever. This can be overridden with one of the + * following options: + * + *
    + *
  • Setting an ending snapshot id with {@code to_snapshot}. + *
  • Setting an ending timestamp (milliseconds) with {@code to_timestamp}. + *
+ * + *

For example: + * + *

{@code
+ * Map config = Map.of(
+ *         "table", table,
+ *         "catalog_name", name,
+ *         "catalog_properties", Map.of(...),
+ *         "config_properties", Map.of(...),
+ *         "from_snapshot", 123456789L,
+ *         "to_timestamp", 987654321L);
+ *
+ * PCollection = pipeline
+ *     .apply(Managed.read(ICEBERG_CDC).withConfig(config))
+ *     .getSinglePCollection();
+ * }
+ * + * Note: If {@code streaming=true} and an end point is set, the pipeline will run in + * streaming mode and shut down automatically after processing the final snapshot. */ @Internal public class IcebergIO { @@ -395,16 +601,39 @@ public IcebergWriteResult expand(PCollection input) { } public static ReadRows readRows(IcebergCatalogConfig catalogConfig) { - return new AutoValue_IcebergIO_ReadRows.Builder().setCatalogConfig(catalogConfig).build(); + return new AutoValue_IcebergIO_ReadRows.Builder() + .setCatalogConfig(catalogConfig) + .setUseCdc(false) + .build(); } @AutoValue public abstract static class ReadRows extends PTransform> { + public enum StartingStrategy { + EARLIEST, + LATEST + } abstract IcebergCatalogConfig getCatalogConfig(); abstract @Nullable TableIdentifier getTableIdentifier(); + abstract boolean getUseCdc(); + + abstract @Nullable Long getFromSnapshot(); + + abstract @Nullable Long getToSnapshot(); + + abstract @Nullable Long getFromTimestamp(); + + abstract @Nullable Long getToTimestamp(); + + abstract @Nullable StartingStrategy getStartingStrategy(); + + abstract @Nullable Boolean getStreaming(); + + abstract @Nullable Duration getPollInterval(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -413,13 +642,61 @@ abstract static class Builder { abstract Builder setTableIdentifier(TableIdentifier identifier); + abstract Builder setUseCdc(boolean useCdc); + + abstract Builder setFromSnapshot(@Nullable Long fromSnapshot); + + abstract Builder setToSnapshot(@Nullable Long toSnapshot); + + abstract Builder setFromTimestamp(@Nullable Long fromTimestamp); + + abstract Builder setToTimestamp(@Nullable Long toTimestamp); + + abstract Builder setStartingStrategy(@Nullable StartingStrategy strategy); + + abstract Builder setStreaming(@Nullable Boolean streaming); + + abstract Builder setPollInterval(@Nullable Duration triggeringFrequency); + abstract ReadRows build(); } + public ReadRows withCdc() { + return toBuilder().setUseCdc(true).build(); + } + public ReadRows from(TableIdentifier tableIdentifier) { return toBuilder().setTableIdentifier(tableIdentifier).build(); } + public ReadRows fromSnapshot(@Nullable Long fromSnapshot) { + return toBuilder().setFromSnapshot(fromSnapshot).build(); + } + + public ReadRows toSnapshot(@Nullable Long toSnapshot) { + return toBuilder().setToSnapshot(toSnapshot).build(); + } + + public ReadRows fromTimestamp(@Nullable Long fromTimestamp) { + return toBuilder().setFromTimestamp(fromTimestamp).build(); + } + + public ReadRows toTimestamp(@Nullable Long toTimestamp) { + return toBuilder().setToTimestamp(toTimestamp).build(); + } + + public ReadRows withPollInterval(Duration pollInterval) { + return toBuilder().setPollInterval(pollInterval).build(); + } + + public ReadRows streaming(@Nullable Boolean streaming) { + return toBuilder().setStreaming(streaming).build(); + } + + public ReadRows withStartingStrategy(@Nullable StartingStrategy strategy) { + return toBuilder().setStartingStrategy(strategy).build(); + } + @Override public PCollection expand(PBegin input) { TableIdentifier tableId = @@ -427,15 +704,29 @@ public PCollection expand(PBegin input) { Table table = getCatalogConfig().catalog().loadTable(tableId); - return input.apply( - Read.from( - new ScanSource( - IcebergScanConfig.builder() - .setCatalogConfig(getCatalogConfig()) - .setScanType(IcebergScanConfig.ScanType.TABLE) - .setTableIdentifier(tableId) - .setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())) - .build()))); + IcebergScanConfig scanConfig = + IcebergScanConfig.builder() + .setCatalogConfig(getCatalogConfig()) + .setScanType(IcebergScanConfig.ScanType.TABLE) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())) + .setFromSnapshotInclusive(getFromSnapshot()) + .setToSnapshot(getToSnapshot()) + .setFromTimestamp(getFromTimestamp()) + .setToTimestamp(getToTimestamp()) + .setStartingStrategy(getStartingStrategy()) + .setStreaming(getStreaming()) + .setPollInterval(getPollInterval()) + .setUseCdc(getUseCdc()) + .build(); + scanConfig.validate(table); + + PTransform> source = + getUseCdc() + ? new IncrementalScanSource(scanConfig) + : Read.from(new ScanSource(scanConfig)); + + return input.apply(source); } } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java index d44149fda08e..35500676ae2f 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProvider.java @@ -17,14 +17,20 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.Configuration; import static org.apache.beam.sdk.util.construction.BeamUrns.getUrn; import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; import java.util.Collections; import java.util.List; +import java.util.Map; import org.apache.beam.model.pipeline.v1.ExternalTransforms; +import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.NoSuchSchemaException; import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; import org.apache.beam.sdk.schemas.transforms.SchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; @@ -32,6 +38,7 @@ import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.Nullable; /** * SchemaTransform implementation for {@link IcebergIO#readRows}. Reads records from Iceberg and @@ -40,11 +47,11 @@ */ @AutoService(SchemaTransformProvider.class) public class IcebergReadSchemaTransformProvider - extends TypedSchemaTransformProvider { + extends TypedSchemaTransformProvider { static final String OUTPUT_TAG = "output"; @Override - protected SchemaTransform from(SchemaTransformConfiguration configuration) { + protected SchemaTransform from(Configuration configuration) { return new IcebergReadSchemaTransform(configuration); } @@ -59,9 +66,9 @@ public String identifier() { } static class IcebergReadSchemaTransform extends SchemaTransform { - private final SchemaTransformConfiguration configuration; + private final Configuration configuration; - IcebergReadSchemaTransform(SchemaTransformConfiguration configuration) { + IcebergReadSchemaTransform(Configuration configuration) { this.configuration = configuration; } @@ -70,7 +77,7 @@ Row getConfigurationRow() { // To stay consistent with our SchemaTransform configuration naming conventions, // we sort lexicographically and convert field names to snake_case return SchemaRegistry.createDefault() - .getToRowFunction(SchemaTransformConfiguration.class) + .getToRowFunction(Configuration.class) .apply(configuration) .sorted() .toSnakeCase(); @@ -91,4 +98,48 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { return PCollectionRowTuple.of(OUTPUT_TAG, output); } } + + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class Configuration { + static Builder builder() { + return new AutoValue_IcebergReadSchemaTransformProvider_Configuration.Builder(); + } + + @SchemaFieldDescription("Identifier of the Iceberg table.") + abstract String getTable(); + + @SchemaFieldDescription("Name of the catalog containing the table.") + @Nullable + abstract String getCatalogName(); + + @SchemaFieldDescription("Properties used to set up the Iceberg catalog.") + @Nullable + abstract Map getCatalogProperties(); + + @SchemaFieldDescription("Properties passed to the Hadoop Configuration.") + @Nullable + abstract Map getConfigProperties(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setTable(String table); + + abstract Builder setCatalogName(String catalogName); + + abstract Builder setCatalogProperties(Map catalogProperties); + + abstract Builder setConfigProperties(Map confProperties); + + abstract Configuration build(); + } + + IcebergCatalogConfig getIcebergCatalog() { + return IcebergCatalogConfig.builder() + .setCatalogName(getCatalogName()) + .setCatalogProperties(getCatalogProperties()) + .setConfigProperties(getConfigProperties()) + .build(); + } + } } 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 60372b172af7..1bdde96b9af3 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 @@ -17,9 +17,16 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + import com.google.auto.value.AutoValue; import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; 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.ImmutableMap; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -27,11 +34,12 @@ import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; +import org.joda.time.Duration; @AutoValue public abstract class IcebergScanConfig implements Serializable { - private transient @MonotonicNonNull Table cachedTable; + private @MonotonicNonNull Schema cachedCdcSchema; public enum ScanType { TABLE, @@ -59,6 +67,14 @@ public Table getTable() { @Pure public abstract Schema getSchema(); + @Pure + public Schema getCdcSchema() { + if (cachedCdcSchema == null) { + cachedCdcSchema = ReadUtils.outputCdcSchema(getSchema()); + } + return cachedCdcSchema; + } + @Pure public abstract @Nullable Expression getFilter(); @@ -92,6 +108,24 @@ public Table getTable() { @Pure public abstract @Nullable String getToSnapshotRef(); + @Pure + public abstract @Nullable Long getFromTimestamp(); + + @Pure + public abstract @Nullable Long getToTimestamp(); + + @Pure + public abstract @Nullable StartingStrategy getStartingStrategy(); + + @Pure + public abstract boolean getUseCdc(); + + @Pure + public abstract @Nullable Boolean getStreaming(); + + @Pure + public abstract @Nullable Duration getPollInterval(); + @Pure public abstract @Nullable String getTag(); @@ -113,6 +147,12 @@ public static Builder builder() { .setFromSnapshotRefExclusive(null) .setToSnapshot(null) .setToSnapshotRef(null) + .setFromTimestamp(null) + .setToTimestamp(null) + .setUseCdc(false) + .setStreaming(null) + .setPollInterval(null) + .setStartingStrategy(null) .setTag(null) .setBranch(null); } @@ -157,10 +197,83 @@ public Builder setTableIdentifier(String... names) { public abstract Builder setToSnapshotRef(@Nullable String ref); + public abstract Builder setFromTimestamp(@Nullable Long timestamp); + + public abstract Builder setToTimestamp(@Nullable Long timestamp); + + public abstract Builder setStartingStrategy(@Nullable StartingStrategy strategy); + + public abstract Builder setUseCdc(boolean useCdc); + + public abstract Builder setStreaming(@Nullable Boolean streaming); + + public abstract Builder setPollInterval(@Nullable Duration pollInterval); + public abstract Builder setTag(@Nullable String tag); public abstract Builder setBranch(@Nullable String branch); public abstract IcebergScanConfig build(); } + + @VisibleForTesting + abstract Builder toBuilder(); + + void validate(Table table) { + // TODO(#34168, ahmedabu98): fill these gaps for the existing batch source + if (!getUseCdc()) { + List invalidOptions = new ArrayList<>(); + if (MoreObjects.firstNonNull(getStreaming(), false)) { + invalidOptions.add("streaming"); + } + if (getPollInterval() != null) { + invalidOptions.add("poll_interval_seconds"); + } + if (getFromTimestamp() != null) { + invalidOptions.add("from_timestamp"); + } + if (getToTimestamp() != null) { + invalidOptions.add("to_timestamp"); + } + if (getFromSnapshotInclusive() != null) { + invalidOptions.add("from_snapshot"); + } + if (getToSnapshot() != null) { + invalidOptions.add("to_snapshot"); + } + if (getStartingStrategy() != null) { + invalidOptions.add("starting_strategy"); + } + if (!invalidOptions.isEmpty()) { + throw new IllegalArgumentException( + error( + "the following options are currently only available when " + + "reading with Managed.ICEBERG_CDC: " + + invalidOptions)); + } + } + + if (getStartingStrategy() != null) { + checkArgument( + getFromTimestamp() == null && getFromSnapshotInclusive() == null, + error( + "'from_timestamp' and 'from_snapshot' are not allowed when 'starting_strategy' is set")); + } + checkArgument( + getFromTimestamp() == null || getFromSnapshotInclusive() == null, + error("only one of 'from_timestamp' or 'from_snapshot' can be set")); + checkArgument( + getToTimestamp() == null || getToSnapshot() == null, + error("only one of 'to_timestamp' or 'to_snapshot' can be set")); + + if (getPollInterval() != null) { + checkArgument( + Boolean.TRUE.equals(getStreaming()), + error("'poll_interval_seconds' can only be set when streaming is true")); + } + } + + private String error(String message) { + return "Invalid source configuration: " + message; + } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java index c33f7d6261e8..801464010c69 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslation.java @@ -23,6 +23,7 @@ import com.google.auto.service.AutoService; import java.util.Map; +import org.apache.beam.sdk.io.iceberg.IcebergCdcReadSchemaTransformProvider.IcebergCdcReadSchemaTransform; import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.construction.PTransformTranslation.TransformPayloadTranslator; @@ -45,6 +46,19 @@ public Row toConfigRow(IcebergReadSchemaTransform transform) { } } + static class IcebergCdcReadSchemaTransformTranslator + extends SchemaTransformPayloadTranslator { + @Override + public SchemaTransformProvider provider() { + return new IcebergCdcReadSchemaTransformProvider(); + } + + @Override + public Row toConfigRow(IcebergCdcReadSchemaTransform transform) { + return transform.getConfigurationRow(); + } + } + @AutoService(TransformPayloadTranslatorRegistrar.class) public static class ReadRegistrar implements TransformPayloadTranslatorRegistrar { @Override @@ -55,6 +69,7 @@ public static class ReadRegistrar implements TransformPayloadTranslatorRegistrar getTransformPayloadTranslators() { return ImmutableMap., TransformPayloadTranslator>builder() .put(IcebergReadSchemaTransform.class, new IcebergReadSchemaTransformTranslator()) + .put(IcebergCdcReadSchemaTransform.class, new IcebergCdcReadSchemaTransformTranslator()) .build(); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java index 6aa830e7fbc6..de61b8f3c6dc 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergWriteSchemaTransformProvider.java @@ -58,7 +58,10 @@ public class IcebergWriteSchemaTransformProvider static final String SNAPSHOTS_TAG = "snapshots"; static final Schema OUTPUT_SCHEMA = - Schema.builder().addStringField("table").addFields(SnapshotInfo.SCHEMA.getFields()).build(); + Schema.builder() + .addStringField("table") + .addFields(SnapshotInfo.getSchema().getFields()) + .build(); @Override public String description() { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java new file mode 100644 index 000000000000..d3e638b067a9 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java @@ -0,0 +1,157 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.util.List; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.GroupIntoBatches; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Redistribute; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.util.ShardedKey; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +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.base.MoreObjects; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.iceberg.Table; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; + +/** + * An Iceberg source that reads a table incrementally using range(s) of table snapshots. The bounded + * source creates a single range, while the unbounded implementation continuously polls for new + * snapshots at the specified interval. + * + *

Outputs CDC Rows with the following schema: + * + *

    + *
  • "record" [Row]: the data record itself + *
  • "operation" [String]: the operation associated with this record + *
+ */ +class IncrementalScanSource extends PTransform> { + // (streaming) We will group files into batches of this size + // Downstream, we create one reader per batch + // TODO(ahmedabu98): should we make this configurable? + private static final long MAX_FILES_BATCH_BYTE_SIZE = 1L << 32; // 4 GB + private static final Duration DEFAULT_POLL_INTERVAL = Duration.standardSeconds(60); + private final IcebergScanConfig scanConfig; + + IncrementalScanSource(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + } + + @Override + public PCollection expand(PBegin input) { + Table table = + TableCache.getRefreshed( + scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); + + PCollection rows = + MoreObjects.firstNonNull(scanConfig.getStreaming(), false) + ? readUnbounded(input) + : readBounded(input, table); + return rows.setRowSchema(ReadUtils.outputCdcSchema(table.schema())); + } + + /** + * Watches for new snapshots and creates tasks for each range. Uses GiB (with auto-sharding) to + * groups tasks in batches of size {@link #MAX_FILES_BATCH_BYTE_SIZE}, then reads from each batch + * using an SDF. + * + *

Output schema is: + * + *

    + *
  • "record": the actual data record + *
  • "operation": the snapshot operation associated with this record (e.g. "append", + * "replace", "delete") + *
+ */ + private PCollection readUnbounded(PBegin input) { + @Nullable + Duration pollInterval = + MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); + return input + .apply("Watch for Snapshots", new WatchForSnapshots(scanConfig, pollInterval)) + .setCoder(KvCoder.of(StringUtf8Coder.of(), ListCoder.of(SnapshotInfo.getCoder()))) + .apply("Redistribute by Source Table", Redistribute.byKey()) + .apply("Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig))) + .setCoder(KvCoder.of(ReadTaskDescriptor.getCoder(), ReadTask.getCoder())) + .apply( + GroupIntoBatches.ofByteSize( + MAX_FILES_BATCH_BYTE_SIZE, ReadTask::getByteSize) + .withMaxBufferingDuration(pollInterval) + .withShardedKey()) + .setCoder( + KvCoder.of( + ShardedKey.Coder.of(ReadTaskDescriptor.getCoder()), + IterableCoder.of(ReadTask.getCoder()))) + .apply( + "Iterable to List", + MapElements.via( + new SimpleFunction< + KV, Iterable>, + KV>>() { + @Override + public KV> apply( + KV, Iterable> input) { + return KV.of(input.getKey().getKey(), Lists.newArrayList(input.getValue())); + } + })) + .apply("Read Rows From Grouped Tasks", ParDo.of(new ReadFromGroupedTasks(scanConfig))); + } + + /** + * Scans a single snapshot range and creates read tasks. Tasks are redistributed and processed + * individually using a regular DoFn. + */ + private PCollection readBounded(PBegin input, Table table) { + checkStateNotNull( + table.currentSnapshot().snapshotId(), + "Table %s does not have any snapshots to read from.", + scanConfig.getTableIdentifier()); + + @Nullable Long from = ReadUtils.getFromSnapshotExclusive(table, scanConfig); + // if no end snapshot is provided, we read up to the current snapshot. + long to = + MoreObjects.firstNonNull( + ReadUtils.getToSnapshot(table, scanConfig), table.currentSnapshot().snapshotId()); + return input + .apply( + "Create Snapshot Range", + Create.of( + KV.of( + scanConfig.getTableIdentifier(), + ReadUtils.snapshotsBetween(table, scanConfig.getTableIdentifier(), from, to)))) + .setCoder(KvCoder.of(StringUtf8Coder.of(), ListCoder.of(SnapshotInfo.getCoder()))) + .apply("Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig))) + .setCoder(KvCoder.of(ReadTaskDescriptor.getCoder(), ReadTask.getCoder())) + .apply(Redistribute.arbitrarily()) + .apply("Read Rows From Tasks", ParDo.of(new ReadFromTasks(scanConfig))); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromGroupedTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromGroupedTasks.java new file mode 100644 index 000000000000..3df7891e9b07 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromGroupedTasks.java @@ -0,0 +1,106 @@ +/* + * 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.io.iceberg; + +import java.io.IOException; +import java.util.List; +import java.util.concurrent.ExecutionException; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Unbounded read implementation. + * + *

An SDF that takes a batch of {@link ReadTask}s. For each task, reads Iceberg {@link Record}s, + * and converts to Beam {@link Row}s. + * + *

The SDF checkpoints after reading each task, and can split the batch of read tasks as needed. + */ +@DoFn.BoundedPerElement +class ReadFromGroupedTasks extends DoFn>, Row> { + private final IcebergScanConfig scanConfig; + private final Counter scanTasksCompleted = + Metrics.counter(ReadFromGroupedTasks.class, "scanTasksCompleted"); + + ReadFromGroupedTasks(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + } + + @ProcessElement + public void process( + @Element KV> element, + RestrictionTracker tracker, + OutputReceiver out) + throws IOException, ExecutionException { + List readTasks = element.getValue(); + Table table = + TableCache.get(scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); + + // SDF can split by the number of read tasks + for (long taskIndex = tracker.currentRestriction().getFrom(); + taskIndex < tracker.currentRestriction().getTo(); + ++taskIndex) { + if (!tracker.tryClaim(taskIndex)) { + return; + } + + ReadTask readTask = readTasks.get((int) taskIndex); + @Nullable String operation = readTask.getOperation(); + FileScanTask task = readTask.getFileScanTask(); + + try (CloseableIterable reader = ReadUtils.createReader(task, table)) { + for (Record record : reader) { + Row row = + Row.withSchema(scanConfig.getCdcSchema()) + .addValue(IcebergUtils.icebergRecordToBeamRow(scanConfig.getSchema(), record)) + .addValue(operation) + .build(); + out.output(row); + } + } + scanTasksCompleted.inc(); + } + } + + @GetInitialRestriction + public OffsetRange getInitialRange(@Element KV> element) { + return new OffsetRange(0, element.getValue().size()); + } + + @GetSize + public double getSize( + @Element KV> element, + @Restriction OffsetRange restriction) { + double size = 0; + List tasks = element.getValue(); + for (ReadTask task : tasks.subList((int) restriction.getFrom(), tasks.size())) { + size += task.getByteSize(); + } + return size; + } +} 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 new file mode 100644 index 000000000000..d85f4202a6e1 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -0,0 +1,69 @@ +/* + * 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.io.iceberg; + +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** + * Bounded read implementation. + * + *

For each {@link ReadTask}, reads Iceberg {@link Record}s, and converts to Beam {@link Row}s. + */ +class ReadFromTasks extends DoFn, Row> { + private final IcebergScanConfig scanConfig; + private final Counter scanTasksCompleted = + Metrics.counter(ReadFromTasks.class, "scanTasksCompleted"); + + ReadFromTasks(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + } + + @ProcessElement + public void process(@Element KV element, OutputReceiver out) + throws IOException, ExecutionException { + ReadTask readTask = element.getValue(); + Table table = + TableCache.get(scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); + + FileScanTask task = readTask.getFileScanTask(); + @Nullable String operation = readTask.getOperation(); + + try (CloseableIterable reader = ReadUtils.createReader(task, table)) { + for (Record record : reader) { + Row row = + Row.withSchema(scanConfig.getCdcSchema()) + .addValue(IcebergUtils.icebergRecordToBeamRow(scanConfig.getSchema(), record)) + .addValue(operation) + .build(); + out.output(row); + } + } + scanTasksCompleted.inc(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTask.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTask.java new file mode 100644 index 000000000000..4a155eae64c3 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTask.java @@ -0,0 +1,82 @@ +/* + * 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.io.iceberg; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaIgnore; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ScanTaskParser; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.Nullable; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +abstract class ReadTask { + private static @MonotonicNonNull SchemaCoder coder; + + static SchemaCoder getCoder() { + if (coder == null) { + try { + coder = SchemaRegistry.createDefault().getSchemaCoder(ReadTask.class); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + return coder; + } + + private transient @MonotonicNonNull FileScanTask cachedFileScanTask; + + static Builder builder() { + return new AutoValue_ReadTask.Builder(); + } + + abstract String getFileScanTaskJson(); + + abstract long getByteSize(); + + abstract @Nullable String getOperation(); + + abstract long getSnapshotTimestampMillis(); + + @SchemaIgnore + FileScanTask getFileScanTask() { + if (cachedFileScanTask == null) { + cachedFileScanTask = ScanTaskParser.fromJson(getFileScanTaskJson(), true); + } + return cachedFileScanTask; + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setFileScanTaskJson(String jsonTask); + + abstract Builder setByteSize(long size); + + abstract Builder setOperation(@Nullable String operation); + + abstract Builder setSnapshotTimestampMillis(long millis); + + abstract ReadTask build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTaskDescriptor.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTaskDescriptor.java new file mode 100644 index 000000000000..b7a9be32aba2 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTaskDescriptor.java @@ -0,0 +1,57 @@ +/* + * 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.io.iceberg; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.NoSuchSchemaException; +import org.apache.beam.sdk.schemas.SchemaCoder; +import org.apache.beam.sdk.schemas.SchemaRegistry; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; + +/** Describes the table a {@link ReadTask} belongs to. */ +@DefaultSchema(AutoValueSchema.class) +@AutoValue +abstract class ReadTaskDescriptor { + private static @MonotonicNonNull SchemaCoder coder; + + static SchemaCoder getCoder() { + if (coder == null) { + try { + coder = SchemaRegistry.createDefault().getSchemaCoder(ReadTaskDescriptor.class); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + return coder; + } + + static Builder builder() { + return new AutoValue_ReadTaskDescriptor.Builder(); + } + + abstract String getTableIdentifierString(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setTableIdentifierString(String table); + + abstract ReadTaskDescriptor build(); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java new file mode 100644 index 000000000000..2516e4db76c3 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java @@ -0,0 +1,242 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.iceberg.util.SnapshotUtil.ancestorsOf; + +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; +import java.util.stream.Collectors; +import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +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.base.MoreObjects; +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.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.IdentityPartitionConverters; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedInputFile; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.parquet.ParquetReader; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PartitionUtil; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; + +/** Helper class for source operations. */ +public class ReadUtils { + // default is 8MB. keep this low to avoid overwhelming memory + static final int MAX_FILE_BUFFER_SIZE = 1 << 20; // 1MB + private static final Collection READ_PROPERTIES_TO_REMOVE = + Sets.newHashSet( + "parquet.read.filter", + "parquet.private.read.filter.predicate", + "parquet.read.support.class", + "parquet.crypto.factory.class"); + static final String OPERATION = "operation"; + static final String RECORD = "record"; + + /** Extracts {@link Row}s after a CDC streaming read. */ + public static PTransform, PCollection> extractRecords() { + return new ExtractRecords(); + } + + public static Schema outputCdcSchema(Schema tableSchema) { + return Schema.builder() + .addRowField(RECORD, tableSchema) + .addNullableStringField(OPERATION) + .build(); + } + + public static Schema outputCdcSchema(org.apache.iceberg.Schema tableSchema) { + return outputCdcSchema(IcebergUtils.icebergSchemaToBeamSchema(tableSchema)); + } + + static ParquetReader createReader(FileScanTask task, Table table) { + String filePath = task.file().path().toString(); + InputFile inputFile; + try (FileIO io = table.io()) { + EncryptedInputFile encryptedInput = + EncryptedFiles.encryptedInput(io.newInputFile(filePath), task.file().keyMetadata()); + inputFile = table.encryption().decrypt(encryptedInput); + } + Map idToConstants = + ReadUtils.constantsMap(task, IdentityPartitionConverters::convertConstant, table.schema()); + + ParquetReadOptions.Builder optionsBuilder; + if (inputFile instanceof HadoopInputFile) { + // remove read properties already set that may conflict with this read + Configuration conf = new Configuration(((HadoopInputFile) inputFile).getConf()); + for (String property : READ_PROPERTIES_TO_REMOVE) { + conf.unset(property); + } + optionsBuilder = HadoopReadOptions.builder(conf); + } else { + optionsBuilder = ParquetReadOptions.builder(); + } + optionsBuilder = + optionsBuilder + .withRange(task.start(), task.start() + task.length()) + .withMaxAllocationInBytes(MAX_FILE_BUFFER_SIZE); + + @Nullable String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping mapping = + nameMapping != null ? NameMappingParser.fromJson(nameMapping) : NameMapping.empty(); + + return new ParquetReader<>( + inputFile, + table.schema(), + optionsBuilder.build(), + fileSchema -> GenericParquetReaders.buildReader(table.schema(), fileSchema, idToConstants), + mapping, + task.residual(), + false, + true); + } + + static Map constantsMap( + FileScanTask task, + BiFunction converter, + org.apache.iceberg.Schema schema) { + PartitionSpec spec = task.spec(); + Set idColumns = spec.identitySourceIds(); + org.apache.iceberg.Schema partitionSchema = TypeUtil.select(schema, idColumns); + boolean projectsIdentityPartitionColumns = !partitionSchema.columns().isEmpty(); + + if (projectsIdentityPartitionColumns) { + return PartitionUtil.constantsMap(task, converter); + } else { + return Collections.emptyMap(); + } + } + + static @Nullable Long getFromSnapshotExclusive(Table table, IcebergScanConfig scanConfig) { + @Nullable StartingStrategy startingStrategy = scanConfig.getStartingStrategy(); + boolean isStreaming = MoreObjects.firstNonNull(scanConfig.getStreaming(), false); + if (startingStrategy == null) { + startingStrategy = isStreaming ? StartingStrategy.LATEST : StartingStrategy.EARLIEST; + } + + // 1. fetch from from_snapshot + @Nullable Long fromSnapshot = scanConfig.getFromSnapshotInclusive(); + // 2. fetch from from_timestamp + @Nullable Long fromTimestamp = scanConfig.getFromTimestamp(); + if (fromTimestamp != null) { + fromSnapshot = SnapshotUtil.oldestAncestorAfter(table, fromTimestamp).snapshotId(); + } + // 3. get current snapshot if starting_strategy is LATEST + if (fromSnapshot == null && startingStrategy.equals(StartingStrategy.LATEST)) { + @Nullable Snapshot currentSnapshot = table.currentSnapshot(); + if (currentSnapshot != null) { + fromSnapshot = currentSnapshot.snapshotId(); + } + } + // incremental append scan can only be configured with an *exclusive* starting snapshot, + // so we need to provide this snapshot's parent id. + if (fromSnapshot != null) { + fromSnapshot = table.snapshot(fromSnapshot).parentId(); + } + + // 4. if snapshot is still null, the scan will default to the oldest snapshot, i.e. EARLIEST + return fromSnapshot; + } + + static @Nullable Long getToSnapshot(Table table, IcebergScanConfig scanConfig) { + // 1. fetch from to_snapshot + @Nullable Long toSnapshot = scanConfig.getToSnapshot(); + // 2. fetch from to_timestamp + @Nullable Long toTimestamp = scanConfig.getToTimestamp(); + if (toTimestamp != null) { + toSnapshot = SnapshotUtil.snapshotIdAsOfTime(table, toTimestamp); + } + + return toSnapshot; + } + + /** + * Returns a list of snapshots in the range (fromSnapshotId, toSnapshotId], ordered + * chronologically. + */ + static List snapshotsBetween( + Table table, String tableIdentifier, @Nullable Long fromSnapshotId, long toSnapshotId) { + long from = MoreObjects.firstNonNull(fromSnapshotId, -1L); + @SuppressWarnings("return") + List snapshotIds = + Lists.newArrayList( + Lists.newArrayList( + ancestorsOf( + toSnapshotId, + snapshotId -> snapshotId != from ? table.snapshot(snapshotId) : null))) + .stream() + .map(s -> SnapshotInfo.fromSnapshot(s, tableIdentifier)) + .sorted(Comparator.comparingLong(SnapshotInfo::getSequenceNumber)) + .collect(Collectors.toList()); + + return snapshotIds; + } + + private static class ExtractRecords extends PTransform, PCollection> { + @Override + public PCollection expand(PCollection input) { + Preconditions.checkArgument( + input.getSchema().hasField(RECORD) + && input.getSchema().getField(RECORD).getType().getTypeName().isCompositeType(), + "PCollection schema must contain a \"%s\" field of type %s. Actual schema: %s", + RECORD, + Schema.TypeName.ROW, + input.getSchema()); + Schema recordSchema = + checkStateNotNull(input.getSchema().getField(RECORD).getType().getRowSchema()); + return input.apply(ParDo.of(new ExtractRecordsDoFn())).setRowSchema(recordSchema); + } + + static class ExtractRecordsDoFn extends DoFn { + @ProcessElement + public void process(@Element Row row, @Timestamp Instant timestamp, OutputReceiver out) { + out.outputWithTimestamp(checkStateNotNull(row.getRow(RECORD)), timestamp); + } + } + } +} 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 c9906618a64d..6ff0a8d17789 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 @@ -21,19 +21,14 @@ import java.io.IOException; import java.util.ArrayDeque; -import java.util.Collections; import java.util.Map; import java.util.NoSuchElementException; import java.util.Queue; -import java.util.Set; -import java.util.function.BiFunction; import javax.annotation.Nullable; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.values.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.avro.Avro; @@ -51,9 +46,6 @@ import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.PartitionUtil; import org.checkerframework.checker.nullness.qual.NonNull; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -127,7 +119,7 @@ public boolean advance() throws IOException { DataFile file = fileTask.file(); InputFile input = decryptor.getInputFile(fileTask); Map idToConstants = - constantsMap(fileTask, IdentityPartitionConverters::convertConstant, project); + ReadUtils.constantsMap(fileTask, IdentityPartitionConverters::convertConstant, project); CloseableIterable iterable; switch (file.format()) { @@ -190,20 +182,6 @@ public boolean advance() throws IOException { return false; } - private Map constantsMap( - FileScanTask task, BiFunction converter, Schema schema) { - PartitionSpec spec = task.spec(); - Set idColumns = spec.identitySourceIds(); - Schema partitionSchema = TypeUtil.select(schema, idColumns); - boolean projectsIdentityPartitionColumns = !partitionSchema.columns().isEmpty(); - - if (projectsIdentityPartitionColumns) { - return PartitionUtil.constantsMap(task, converter); - } else { - return Collections.emptyMap(); - } - } - @Override public Row getCurrent() throws NoSuchElementException { if (current == null) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java deleted file mode 100644 index 6e7a12aa15af..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.io.iceberg; - -import com.google.auto.value.AutoValue; -import java.util.Map; -import org.apache.beam.sdk.schemas.AutoValueSchema; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; -import org.checkerframework.checker.nullness.qual.Nullable; - -@DefaultSchema(AutoValueSchema.class) -@AutoValue -public abstract class SchemaTransformConfiguration { - public static Builder builder() { - return new AutoValue_SchemaTransformConfiguration.Builder(); - } - - @SchemaFieldDescription("Identifier of the Iceberg table.") - public abstract String getTable(); - - @SchemaFieldDescription("Name of the catalog containing the table.") - @Nullable - public abstract String getCatalogName(); - - @SchemaFieldDescription("Properties used to set up the Iceberg catalog.") - @Nullable - public abstract Map getCatalogProperties(); - - @SchemaFieldDescription("Properties passed to the Hadoop Configuration.") - @Nullable - public abstract Map getConfigProperties(); - - @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setTable(String table); - - public abstract Builder setCatalogName(String catalogName); - - public abstract Builder setCatalogProperties(Map catalogProperties); - - public abstract Builder setConfigProperties(Map confProperties); - - public abstract SchemaTransformConfiguration build(); - } - - public IcebergCatalogConfig getIcebergCatalog() { - return IcebergCatalogConfig.builder() - .setCatalogName(getCatalogName()) - .setCatalogProperties(getCatalogProperties()) - .setConfigProperties(getConfigProperties()) - .build(); - } -} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java index f81831d9ea94..aa19ca1b2710 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + import com.google.auto.value.AutoValue; import java.util.Map; import javax.annotation.Nullable; @@ -26,8 +28,12 @@ import org.apache.beam.sdk.schemas.SchemaCoder; import org.apache.beam.sdk.schemas.SchemaRegistry; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaIgnore; 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.iceberg.Snapshot; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; /** * This is an AutoValue representation of an Iceberg {@link Snapshot}. @@ -39,7 +45,12 @@ @AutoValue public abstract class SnapshotInfo { public static SnapshotInfo fromSnapshot(Snapshot snapshot) { + return fromSnapshot(snapshot, null); + } + + public static SnapshotInfo fromSnapshot(Snapshot snapshot, @Nullable String tableIdentifier) { return SnapshotInfo.builder() + .setTableIdentifierString(tableIdentifier) .setSequenceNumber(snapshot.sequenceNumber()) .setSnapshotId(snapshot.snapshotId()) .setParentId(snapshot.parentId()) @@ -63,23 +74,30 @@ public Row toRow() { } } - public static final SchemaCoder CODER; - public static final Schema SCHEMA; + private static @MonotonicNonNull SchemaCoder coder; + private static @MonotonicNonNull Schema schema; - static { - try { - SchemaRegistry registry = SchemaRegistry.createDefault(); - CODER = registry.getSchemaCoder(SnapshotInfo.class); - SCHEMA = registry.getSchema(SnapshotInfo.class).sorted().toSnakeCase(); - } catch (NoSuchSchemaException e) { - throw new RuntimeException(e); + static SchemaCoder getCoder() { + if (coder == null) { + initSchemaAndCoder(); } + return checkStateNotNull(coder); } + private transient @MonotonicNonNull TableIdentifier cachedTableIdentifier; + public static Builder builder() { return new AutoValue_SnapshotInfo.Builder(); } + @SchemaIgnore + public TableIdentifier getTableIdentifier() { + if (cachedTableIdentifier == null) { + cachedTableIdentifier = TableIdentifier.parse(checkStateNotNull(getTableIdentifierString())); + } + return cachedTableIdentifier; + } + public abstract long getSequenceNumber(); public abstract long getSnapshotId(); @@ -96,6 +114,8 @@ public static Builder builder() { public abstract @Nullable Integer getSchemaId(); + public abstract @Nullable String getTableIdentifierString(); + @AutoValue.Builder public abstract static class Builder { public abstract Builder setSequenceNumber(long sequenceNumber); @@ -114,6 +134,26 @@ public abstract static class Builder { public abstract Builder setSchemaId(Integer schemaId); + abstract Builder setTableIdentifierString(@Nullable String table); + public abstract SnapshotInfo build(); } + + @VisibleForTesting + static Schema getSchema() { + if (schema == null) { + initSchemaAndCoder(); + } + return checkStateNotNull(schema); + } + + private static void initSchemaAndCoder() { + try { + SchemaRegistry registry = SchemaRegistry.createDefault(); + coder = registry.getSchemaCoder(SnapshotInfo.class); + schema = registry.getSchema(SnapshotInfo.class).sorted().toSnakeCase(); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java new file mode 100644 index 000000000000..448f3efa805d --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java @@ -0,0 +1,60 @@ +/* + * 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.io.iceberg; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Utility to fetch and cache Iceberg {@link Table}s. */ +class TableCache { + private static final Cache CACHE = + CacheBuilder.newBuilder().expireAfterWrite(3, TimeUnit.MINUTES).build(); + + static Table get(TableIdentifier identifier, Catalog catalog) { + try { + return CACHE.get(identifier, () -> catalog.loadTable(identifier)); + } catch (ExecutionException e) { + throw new RuntimeException( + "Encountered a problem fetching table " + identifier + " from cache.", e); + } + } + + static Table get(String identifier, Catalog catalog) { + return get(TableIdentifier.parse(identifier), catalog); + } + + static Table getRefreshed(TableIdentifier identifier, Catalog catalog) { + @Nullable Table table = CACHE.getIfPresent(identifier); + if (table == null) { + return get(identifier, catalog); + } + table.refresh(); + CACHE.put(identifier, table); + return table; + } + + static Table getRefreshed(String identifier, Catalog catalog) { + return getRefreshed(TableIdentifier.parse(identifier), catalog); + } +} diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WatchForSnapshots.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WatchForSnapshots.java new file mode 100644 index 000000000000..50dc241b39ed --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WatchForSnapshots.java @@ -0,0 +1,132 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.transforms.Watch.Growth.PollResult; + +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Gauge; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.Watch; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +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.base.Objects; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Keeps watch over an Iceberg table and continuously outputs a range of snapshots, at the specified + * interval. + * + *

A downstream transform will create a list of read tasks for each range. + */ +class WatchForSnapshots extends PTransform>>> { + private static final Logger LOG = LoggerFactory.getLogger(WatchForSnapshots.class); + private final Duration pollInterval; + private final IcebergScanConfig scanConfig; + + WatchForSnapshots(IcebergScanConfig scanConfig, Duration pollInterval) { + this.pollInterval = pollInterval; + this.scanConfig = scanConfig; + } + + @Override + public PCollection>> expand(PBegin input) { + return input + .apply(Create.of(scanConfig.getTableIdentifier())) + .apply( + "Watch for Snapshots", + Watch.growthOf(new SnapshotPollFn(scanConfig)) + .withPollInterval(pollInterval) + .withOutputCoder(ListCoder.of(SnapshotInfo.getCoder()))); + } + + private static class SnapshotPollFn extends Watch.Growth.PollFn> { + private final Gauge latestSnapshot = Metrics.gauge(SnapshotPollFn.class, "latestSnapshot"); + private final Counter snapshotsObserved = + Metrics.counter(SnapshotPollFn.class, "snapshotsObserved"); + private final IcebergScanConfig scanConfig; + private @Nullable Long fromSnapshotId; + + SnapshotPollFn(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + } + + @Override + public PollResult> apply(String tableIdentifier, Context c) { + // fetch a fresh table to catch updated snapshots + Table table = + TableCache.getRefreshed(tableIdentifier, scanConfig.getCatalogConfig().catalog()); + @Nullable Long userSpecifiedToSnapshot = ReadUtils.getToSnapshot(table, scanConfig); + boolean isComplete = userSpecifiedToSnapshot != null; + if (fromSnapshotId == null) { + fromSnapshotId = ReadUtils.getFromSnapshotExclusive(table, scanConfig); + } + + Snapshot currentSnapshot = table.currentSnapshot(); + if (currentSnapshot == null || Objects.equal(currentSnapshot.snapshotId(), fromSnapshotId)) { + // no new snapshots since last poll. return empty result. + return getPollResult(null, isComplete); + } + Long currentSnapshotId = currentSnapshot.snapshotId(); + + // if no upper bound is specified, we poll up to the current snapshot + long toSnapshotId = MoreObjects.firstNonNull(userSpecifiedToSnapshot, currentSnapshotId); + latestSnapshot.set(toSnapshotId); + + List snapshots = + ReadUtils.snapshotsBetween(table, tableIdentifier, fromSnapshotId, toSnapshotId); + fromSnapshotId = currentSnapshotId; + return getPollResult(snapshots, isComplete); + } + + /** Returns an appropriate PollResult based on the requested boundedness. */ + private PollResult> getPollResult( + @Nullable List snapshots, boolean isComplete) { + List>> timestampedSnapshots = new ArrayList<>(1); + if (snapshots != null) { + // watermark based on the oldest observed snapshot in this poll interval + Instant watermark = Instant.ofEpochMilli(snapshots.get(0).getTimestampMillis()); + timestampedSnapshots.add(TimestampedValue.of(snapshots, watermark)); + LOG.info( + "New poll fetched {} snapshots: {}", + snapshots.size(), + snapshots.stream().map(SnapshotInfo::getSnapshotId).collect(Collectors.toList())); + snapshotsObserved.inc(snapshots.size()); + } + + return isComplete + ? PollResult.complete(timestampedSnapshots) // stop at specified snapshot + : PollResult.incomplete(timestampedSnapshots); // continue forever + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProviderTest.java new file mode 100644 index 000000000000..ff00da8d609d --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProviderTest.java @@ -0,0 +1,173 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.io.iceberg.IcebergCdcReadSchemaTransformProvider.Configuration; +import static org.apache.beam.sdk.values.PCollection.IsBounded.BOUNDED; +import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertEquals; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.beam.sdk.managed.Managed; +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.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.yaml.snakeyaml.Yaml; + +/** Tests for {@link IcebergCdcReadSchemaTransformProvider}. */ +public class IcebergCdcReadSchemaTransformProviderTest { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + + @Rule public TestPipeline testPipeline = TestPipeline.create(); + + @Test + public void testBuildTransformWithRow() { + Map properties = new HashMap<>(); + properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); + properties.put("warehouse", "test_location"); + + Row config = + Row.withSchema(new IcebergCdcReadSchemaTransformProvider().configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalog_name", "test-name") + .withFieldValue("catalog_properties", properties) + .withFieldValue("from_snapshot", 123L) + .withFieldValue("to_snapshot", 456L) + .withFieldValue("from_timestamp", 123L) + .withFieldValue("to_timestamp", 456L) + .withFieldValue("starting_strategy", "earliest") + .withFieldValue("poll_interval_seconds", 789) + .build(); + + new IcebergCdcReadSchemaTransformProvider().from(config); + } + + @Test + public void testSimpleScan() throws Exception { + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + TableIdentifier tableId = TableIdentifier.parse(identifier); + + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + + List> expectedRecords = warehouse.commitData(simpleTable); + + Map properties = new HashMap<>(); + properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); + properties.put("warehouse", warehouse.location); + + Configuration.Builder readConfigBuilder = + Configuration.builder() + .setTable(identifier) + .setCatalogName("name") + .setCatalogProperties(properties) + .setStartingStrategy("earliest") + .setToSnapshot(simpleTable.currentSnapshot().snapshotId()); + + final List expectedRows = + expectedRecords.stream() + .flatMap(List::stream) + .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) + .collect(Collectors.toList()); + + PCollection output = + PCollectionRowTuple.empty(testPipeline) + .apply(new IcebergCdcReadSchemaTransformProvider().from(readConfigBuilder.build())) + .getSinglePCollection(); + + assertThat(output.isBounded(), equalTo(BOUNDED)); + PAssert.that(output) + .satisfies( + (Iterable rows) -> { + for (Row row : rows) { + assertEquals(DataOperations.APPEND, row.getString(ReadUtils.OPERATION)); + } + return null; + }); + PAssert.that(output.apply(ReadUtils.extractRecords())).containsInAnyOrder(expectedRows); + + testPipeline.run(); + } + + @Test + public void testStreamingReadUsingManagedTransform() throws Exception { + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + TableIdentifier tableId = TableIdentifier.parse(identifier); + + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + + List> expectedRecords = warehouse.commitData(simpleTable).subList(3, 9); + List snapshots = Lists.newArrayList(simpleTable.snapshots()); + long second = snapshots.get(1).snapshotId(); + long third = snapshots.get(2).snapshotId(); + + String yamlConfig = + String.format( + "table: %s\n" + + "catalog_name: test-name\n" + + "catalog_properties: \n" + + " type: %s\n" + + " warehouse: %s\n" + + "from_snapshot: %s\n" + + "to_snapshot: %s\n" + + "streaming: true", + identifier, CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, warehouse.location, second, third); + + final List expectedRows = + expectedRecords.stream() + .flatMap(List::stream) + .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) + .collect(Collectors.toList()); + + Map configMap = new Yaml().load(yamlConfig); + PCollection output = + testPipeline + .apply(Managed.read(Managed.ICEBERG_CDC).withConfig(configMap)) + .getSinglePCollection() + .apply(ReadUtils.extractRecords()); + + assertThat(output.isBounded(), equalTo(UNBOUNDED)); + PAssert.that(output).containsInAnyOrder(expectedRows); + + testPipeline.run(); + } +} 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 0406ff31e61e..6680a15f2dae 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 @@ -17,12 +17,17 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.io.iceberg.ReadUtils.OPERATION; import static org.apache.beam.sdk.io.iceberg.TestFixtures.createRecord; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.Assert.assertEquals; +import static org.junit.Assume.assumeTrue; import java.io.File; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.UUID; @@ -30,26 +35,31 @@ import java.util.stream.Stream; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; -import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; 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.transforms.DoFn; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PBegin; 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.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DataOperations; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Metrics; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; @@ -63,16 +73,22 @@ import org.apache.iceberg.types.Types; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -@RunWith(JUnit4.class) +@RunWith(Parameterized.class) public class IcebergIOReadTest { private static final Logger LOG = LoggerFactory.getLogger(IcebergIOReadTest.class); @@ -82,16 +98,110 @@ public class IcebergIOReadTest { @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); @Rule public TestPipeline testPipeline = TestPipeline.create(); + @Rule public TestName testName = new TestName(); + @Rule public transient ExpectedException thrown = ExpectedException.none(); - static class PrintRow extends DoFn { + @Parameters + public static Iterable data() { + return Arrays.asList(new Object[][] {{false}, {true}}); + } - @ProcessElement - public void process(@Element Row row, OutputReceiver output) throws Exception { - LOG.info("Got row {}", row); - output.output(row); + // TODO(#34168, ahmedabu98): Update tests when we close feature gaps between regular and cdc + // sources + @Parameter(0) + public boolean useIncrementalScan; + + static class PrintRow extends PTransform, PCollection> { + + @Override + public PCollection expand(PCollection input) { + Schema inputSchema = input.getSchema(); + + return input + .apply( + ParDo.of( + new DoFn() { + @ProcessElement + public void process(@Element Row row, OutputReceiver output) { + LOG.info("Got row {}", row); + output.output(row); + } + })) + .setRowSchema(inputSchema); } } + @Test + public void testFailWhenBothStartingSnapshotAndTimestampAreSet() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + warehouse.createTable(tableId, TestFixtures.SCHEMA); + IcebergIO.ReadRows read = + IcebergIO.readRows(catalogConfig()) + .from(tableId) + .withCdc() + .fromSnapshot(123L) + .fromTimestamp(123L); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid source configuration: only one of 'from_timestamp' or 'from_snapshot' can be set"); + read.expand(PBegin.in(testPipeline)); + } + + @Test + public void testFailWhenBothEndingSnapshotAndTimestampAreSet() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + warehouse.createTable(tableId, TestFixtures.SCHEMA); + IcebergIO.ReadRows read = + IcebergIO.readRows(catalogConfig()) + .withCdc() + .from(tableId) + .toSnapshot(123L) + .toTimestamp(123L); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid source configuration: only one of 'to_timestamp' or 'to_snapshot' can be set"); + read.expand(PBegin.in(testPipeline)); + } + + @Test + public void testFailWhenStartingPointAndStartingStrategyAreSet() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + warehouse.createTable(tableId, TestFixtures.SCHEMA); + IcebergIO.ReadRows read = + IcebergIO.readRows(catalogConfig()) + .withCdc() + .from(tableId) + .fromSnapshot(123L) + .withStartingStrategy(StartingStrategy.EARLIEST); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid source configuration: 'from_timestamp' and 'from_snapshot' are not allowed when 'starting_strategy' is set"); + read.expand(PBegin.in(testPipeline)); + } + + @Test + public void testFailWhenPollIntervalIsSetOnBatchRead() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + warehouse.createTable(tableId, TestFixtures.SCHEMA); + IcebergIO.ReadRows read = + IcebergIO.readRows(catalogConfig()) + .withCdc() + .from(tableId) + .withPollInterval(Duration.standardSeconds(5)); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid source configuration: 'poll_interval_seconds' can only be set when streaming is true"); + read.expand(PBegin.in(testPipeline)); + } + @Test public void testSimpleScan() throws Exception { TableIdentifier tableId = @@ -99,45 +209,24 @@ public void testSimpleScan() throws Exception { Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file1s1.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT1)) - .appendFile( - warehouse.writeRecords( - "file2s1.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT1)) - .appendFile( - warehouse.writeRecords( - "file3s1.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT1)) - .commit(); + List> expectedRecords = warehouse.commitData(simpleTable); + + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId); + if (useIncrementalScan) { + read = read.withCdc().toSnapshot(simpleTable.currentSnapshot().snapshotId()); + } final List expectedRows = - Stream.of( - TestFixtures.FILE1SNAPSHOT1, - TestFixtures.FILE2SNAPSHOT1, - TestFixtures.FILE3SNAPSHOT1) + expectedRecords.stream() .flatMap(List::stream) .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) .collect(Collectors.toList()); - Map catalogProps = - ImmutableMap.builder() - .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .put("warehouse", warehouse.location) - .build(); - - IcebergCatalogConfig catalogConfig = - IcebergCatalogConfig.builder() - .setCatalogName("name") - .setCatalogProperties(catalogProps) - .build(); + PCollection output = testPipeline.apply(read).apply(new PrintRow()); - PCollection output = - testPipeline - .apply(IcebergIO.readRows(catalogConfig).from(tableId)) - .apply(ParDo.of(new PrintRow())) - .setCoder(RowCoder.of(IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); + if (useIncrementalScan) { + output = output.apply(ReadUtils.extractRecords()); + } PAssert.that(output) .satisfies( @@ -189,23 +278,15 @@ public void testIdentityColumnScan() throws Exception { .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) .collect(Collectors.toList()); - Map catalogProps = - ImmutableMap.builder() - .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .put("warehouse", warehouse.location) - .build(); - - IcebergCatalogConfig catalogConfig = - IcebergCatalogConfig.builder() - .setCatalogName("name") - .setCatalogProperties(catalogProps) - .build(); + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId); + if (useIncrementalScan) { + read = read.withCdc().toSnapshot(simpleTable.currentSnapshot().snapshotId()); + } + PCollection output = testPipeline.apply(read).apply(new PrintRow()); - PCollection output = - testPipeline - .apply(IcebergIO.readRows(catalogConfig).from(tableId)) - .apply(ParDo.of(new PrintRow())) - .setCoder(RowCoder.of(IcebergUtils.icebergSchemaToBeamSchema(simpleTable.schema()))); + if (useIncrementalScan) { + output = output.apply(ReadUtils.extractRecords()); + } PAssert.that(output) .satisfies( @@ -310,23 +391,11 @@ public void testNameMappingScan() throws Exception { simpleTable.newFastAppend().appendFile(dataFile).commit(); - Map catalogProps = - ImmutableMap.builder() - .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .put("warehouse", warehouse.location) - .build(); - - IcebergCatalogConfig catalogConfig = - IcebergCatalogConfig.builder() - .setCatalogName("name") - .setCatalogProperties(catalogProps) - .build(); - - PCollection output = - testPipeline - .apply(IcebergIO.readRows(catalogConfig).from(tableId)) - .apply(ParDo.of(new PrintRow())) - .setCoder(RowCoder.of(beamSchema)); + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId); + if (useIncrementalScan) { + read = read.withCdc().toSnapshot(simpleTable.currentSnapshot().snapshotId()); + } + PCollection output = testPipeline.apply(read).apply(new PrintRow()); final Row[] expectedRows = recordData.stream() @@ -334,6 +403,9 @@ public void testNameMappingScan() throws Exception { .map(record -> IcebergUtils.icebergRecordToBeamRow(beamSchema, record)) .toArray(Row[]::new); + if (useIncrementalScan) { + output = output.apply(ReadUtils.extractRecords()); + } PAssert.that(output) .satisfies( (Iterable rows) -> { @@ -344,6 +416,142 @@ public void testNameMappingScan() throws Exception { testPipeline.run(); } + @Test + public void testBatchDefaultsToEarliestStartingStrategy() throws IOException { + runWithStartingStrategy(null, false); + } + + @Test + public void testStreamingDefaultsToLatestStartingStrategy() throws IOException { + runWithStartingStrategy(null, true); + } + + @Test + public void testUseLatestStartingStrategyWithBatch() throws IOException { + runWithStartingStrategy(StartingStrategy.LATEST, false); + } + + @Test + public void testUseEarliestStartingStrategyWithStreaming() throws IOException { + runWithStartingStrategy(StartingStrategy.EARLIEST, true); + } + + @Test + public void testStreamingReadBetweenSnapshots() throws IOException { + runReadWithBoundary(false, true); + } + + @Test + public void testBatchReadBetweenSnapshots() throws IOException { + runReadWithBoundary(false, false); + } + + @Test + public void testStreamingReadBetweenTimestamps() throws IOException { + runReadWithBoundary(false, true); + } + + @Test + public void testBatchReadBetweenTimestamps() throws IOException { + runReadWithBoundary(false, false); + } + + public void runWithStartingStrategy(@Nullable StartingStrategy strategy, boolean streaming) + throws IOException { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + + List> expectedRecords = warehouse.commitData(simpleTable); + if ((strategy == StartingStrategy.LATEST) || (streaming && strategy == null)) { + expectedRecords = expectedRecords.subList(9, 12); + } + final List expectedRows = + expectedRecords.stream() + .flatMap(List::stream) + .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) + .collect(Collectors.toList()); + + IcebergIO.ReadRows readRows = + IcebergIO.readRows(catalogConfig()) + .from(tableId) + .withCdc() + .streaming(streaming) + .toSnapshot(simpleTable.currentSnapshot().snapshotId()); + if (strategy != null) { + readRows = readRows.withStartingStrategy(strategy); + } + + PCollection output = testPipeline.apply(readRows); + PAssert.that(output) + .satisfies( + rows -> { + for (Row row : rows) { + assertEquals(DataOperations.APPEND, checkStateNotNull(row.getString(OPERATION))); + } + return null; + }); + PCollection rows = output.apply(ReadUtils.extractRecords()); + PCollection.IsBounded expectedBoundedness = + streaming ? PCollection.IsBounded.UNBOUNDED : PCollection.IsBounded.BOUNDED; + assertEquals(expectedBoundedness, rows.isBounded()); + + PAssert.that(rows).containsInAnyOrder(expectedRows); + testPipeline.run().waitUntilFinish(); + } + + public void runReadWithBoundary(boolean useSnapshotBoundary, boolean streaming) + throws IOException { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + + // only read data committed in the second and third snapshots + List> expectedRecords = warehouse.commitData(simpleTable).subList(3, 9); + + final List expectedRows = + expectedRecords.stream() + .flatMap(List::stream) + .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) + .collect(Collectors.toList()); + + List snapshots = Lists.newArrayList(simpleTable.snapshots()); + Snapshot secondSnapshot = snapshots.get(1); + Snapshot thirdSnapshot = snapshots.get(2); + + IcebergIO.ReadRows readRows = + IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).streaming(streaming); + + if (useSnapshotBoundary) { + readRows = + readRows.fromSnapshot(secondSnapshot.snapshotId()).toSnapshot(thirdSnapshot.snapshotId()); + } else { // use timestamp boundary + readRows = + readRows + .fromTimestamp(secondSnapshot.timestampMillis() - 1) + .toTimestamp(thirdSnapshot.timestampMillis() + 1); + } + + PCollection output = testPipeline.apply(readRows).apply(new PrintRow()); + PAssert.that(output) + .satisfies( + rows -> { + for (Row row : rows) { + assertEquals(DataOperations.APPEND, checkStateNotNull(row.getString(OPERATION))); + } + return null; + }); + PCollection rows = output.apply(ReadUtils.extractRecords()); + PCollection.IsBounded expectedBoundedness = + streaming ? PCollection.IsBounded.UNBOUNDED : PCollection.IsBounded.BOUNDED; + assertEquals(expectedBoundedness, rows.isBounded()); + + PAssert.that(rows).containsInAnyOrder(expectedRows); + testPipeline.run(); + } + @SuppressWarnings("unchecked") public static GenericRecord avroGenericRecord( org.apache.avro.Schema schema, Map values) { @@ -396,4 +604,17 @@ private static File createTempFile() throws IOException { boolean unused = tempFile.delete(); return tempFile; } + + private IcebergCatalogConfig catalogConfig() { + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); + + return IcebergCatalogConfig.builder() + .setCatalogName("name") + .setCatalogProperties(catalogProps) + .build(); + } } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java index 0311c31da405..78d48aacf2b7 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergReadSchemaTransformProviderTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.Configuration; import static org.apache.beam.sdk.io.iceberg.IcebergReadSchemaTransformProvider.OUTPUT_TAG; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -26,7 +27,6 @@ import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; -import java.util.stream.Stream; import org.apache.beam.sdk.managed.Managed; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.testing.PAssert; @@ -37,12 +37,14 @@ import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.yaml.snakeyaml.Yaml; +/** Tests for {@link IcebergReadSchemaTransformProvider}. */ public class IcebergReadSchemaTransformProviderTest { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -57,14 +59,14 @@ public void testBuildTransformWithRow() { properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", "test_location"); - Row transformConfigRow = + Row config = Row.withSchema(new IcebergReadSchemaTransformProvider().configurationSchema()) .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", properties) .build(); - new IcebergReadSchemaTransformProvider().from(transformConfigRow); + new IcebergReadSchemaTransformProvider().from(config); } @Test @@ -75,42 +77,27 @@ public void testSimpleScan() throws Exception { Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file1s1.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT1)) - .appendFile( - warehouse.writeRecords( - "file2s1.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT1)) - .appendFile( - warehouse.writeRecords( - "file3s1.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT1)) - .commit(); - - final List expectedRows = - Stream.of( - TestFixtures.FILE1SNAPSHOT1, - TestFixtures.FILE2SNAPSHOT1, - TestFixtures.FILE3SNAPSHOT1) - .flatMap(List::stream) - .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) - .collect(Collectors.toList()); + List> expectedRecords = warehouse.commitData(simpleTable); Map properties = new HashMap<>(); properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", warehouse.location); - SchemaTransformConfiguration readConfig = - SchemaTransformConfiguration.builder() + Configuration.Builder readConfigBuilder = + Configuration.builder() .setTable(identifier) .setCatalogName("name") - .setCatalogProperties(properties) - .build(); + .setCatalogProperties(properties); + + final List expectedRows = + expectedRecords.stream() + .flatMap(List::stream) + .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) + .collect(Collectors.toList()); PCollection output = PCollectionRowTuple.empty(testPipeline) - .apply(new IcebergReadSchemaTransformProvider().from(readConfig)) + .apply(new IcebergReadSchemaTransformProvider().from(readConfigBuilder.build())) .get(OUTPUT_TAG); PAssert.that(output) @@ -131,27 +118,7 @@ public void testReadUsingManagedTransform() throws Exception { Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file1s1.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT1)) - .appendFile( - warehouse.writeRecords( - "file2s1.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT1)) - .appendFile( - warehouse.writeRecords( - "file3s1.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT1)) - .commit(); - - final List expectedRows = - Stream.of( - TestFixtures.FILE1SNAPSHOT1, - TestFixtures.FILE2SNAPSHOT1, - TestFixtures.FILE3SNAPSHOT1) - .flatMap(List::stream) - .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) - .collect(Collectors.toList()); + List> expectedRecords = warehouse.commitData(simpleTable); String yamlConfig = String.format( @@ -161,8 +128,14 @@ public void testReadUsingManagedTransform() throws Exception { + " type: %s\n" + " warehouse: %s", identifier, CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, warehouse.location); - Map configMap = new Yaml().load(yamlConfig); + final List expectedRows = + expectedRecords.stream() + .flatMap(List::stream) + .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) + .collect(Collectors.toList()); + + Map configMap = new Yaml().load(yamlConfig); PCollection output = testPipeline .apply(Managed.read(Managed.ICEBERG).withConfig(configMap)) diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java index 18e5f54e299f..a3217503564c 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergSchemaTransformTranslationTest.java @@ -34,6 +34,8 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.iceberg.IcebergCdcReadSchemaTransformProvider.IcebergCdcReadSchemaTransform; +import org.apache.beam.sdk.io.iceberg.IcebergSchemaTransformTranslation.IcebergCdcReadSchemaTransformTranslator; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.SchemaTranslation; @@ -65,6 +67,8 @@ public class IcebergSchemaTransformTranslationTest { new IcebergWriteSchemaTransformProvider(); static final IcebergReadSchemaTransformProvider READ_PROVIDER = new IcebergReadSchemaTransformProvider(); + static final IcebergCdcReadSchemaTransformProvider READ_CDC_PROVIDER = + new IcebergCdcReadSchemaTransformProvider(); private static final Map CATALOG_PROPERTIES = ImmutableMap.builder() @@ -73,18 +77,39 @@ public class IcebergSchemaTransformTranslationTest { .build(); private static final Map CONFIG_PROPERTIES = ImmutableMap.builder().put("key", "value").put("key2", "value2").build(); + private static final Row WRITE_CONFIG_ROW = + Row.withSchema(WRITE_PROVIDER.configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalog_name", "test-name") + .withFieldValue("catalog_properties", CATALOG_PROPERTIES) + .withFieldValue("config_properties", CONFIG_PROPERTIES) + .withFieldValue("keep", Collections.singletonList("str")) + .build(); + + private static final Row READ_CONFIG_ROW = + Row.withSchema(READ_PROVIDER.configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalog_name", "test-name") + .withFieldValue("catalog_properties", CATALOG_PROPERTIES) + .withFieldValue("config_properties", CONFIG_PROPERTIES) + .build(); + + private static final Row READ_CDC_CONFIG_ROW = + Row.withSchema(READ_CDC_PROVIDER.configurationSchema()) + .withFieldValue("table", "test_table_identifier") + .withFieldValue("catalog_name", "test-name") + .withFieldValue("catalog_properties", CATALOG_PROPERTIES) + .withFieldValue("config_properties", CONFIG_PROPERTIES) + .withFieldValue("from_snapshot", 123L) + .withFieldValue("to_timestamp", 456L) + .withFieldValue("poll_interval_seconds", 123) + .withFieldValue("streaming", true) + .build(); @Test public void testReCreateWriteTransformFromRow() { - Row transformConfigRow = - Row.withSchema(WRITE_PROVIDER.configurationSchema()) - .withFieldValue("table", "test_table_identifier") - .withFieldValue("catalog_name", "test-name") - .withFieldValue("catalog_properties", CATALOG_PROPERTIES) - .withFieldValue("config_properties", CONFIG_PROPERTIES) - .build(); IcebergWriteSchemaTransform writeTransform = - (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(transformConfigRow); + (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(WRITE_CONFIG_ROW); IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator translator = new IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator(); @@ -93,7 +118,7 @@ public void testReCreateWriteTransformFromRow() { IcebergWriteSchemaTransform writeTransformFromRow = translator.fromConfigRow(row, PipelineOptionsFactory.create()); - assertEquals(transformConfigRow, writeTransformFromRow.getConfigurationRow()); + assertEquals(WRITE_CONFIG_ROW, writeTransformFromRow.getConfigurationRow()); } @Test @@ -108,16 +133,8 @@ public void testWriteTransformProtoTranslation() Collections.singletonList(Row.withSchema(inputSchema).addValue("a").build()))) .setRowSchema(inputSchema); - Row transformConfigRow = - Row.withSchema(WRITE_PROVIDER.configurationSchema()) - .withFieldValue("table", "test_identifier") - .withFieldValue("catalog_name", "test-name") - .withFieldValue("catalog_properties", CATALOG_PROPERTIES) - .withFieldValue("config_properties", CONFIG_PROPERTIES) - .build(); - IcebergWriteSchemaTransform writeTransform = - (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(transformConfigRow); + (IcebergWriteSchemaTransform) WRITE_PROVIDER.from(WRITE_CONFIG_ROW); PCollectionRowTuple.of(INPUT_TAG, input).apply(writeTransform); // Then translate the pipeline to a proto and extract IcebergWriteSchemaTransform proto @@ -146,7 +163,7 @@ public void testWriteTransformProtoTranslation() assertEquals(WRITE_PROVIDER.configurationSchema(), schemaFromSpec); Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); - assertEquals(transformConfigRow, rowFromSpec); + assertEquals(WRITE_CONFIG_ROW, rowFromSpec); // Use the information in the proto to recreate the IcebergWriteSchemaTransform IcebergSchemaTransformTranslation.IcebergWriteSchemaTransformTranslator translator = @@ -154,22 +171,13 @@ public void testWriteTransformProtoTranslation() IcebergWriteSchemaTransform writeTransformFromSpec = translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); - assertEquals(transformConfigRow, writeTransformFromSpec.getConfigurationRow()); + assertEquals(WRITE_CONFIG_ROW, writeTransformFromSpec.getConfigurationRow()); } @Test public void testReCreateReadTransformFromRow() { - // setting a subset of fields here. - Row transformConfigRow = - Row.withSchema(READ_PROVIDER.configurationSchema()) - .withFieldValue("table", "test_table_identifier") - .withFieldValue("catalog_name", "test-name") - .withFieldValue("catalog_properties", CATALOG_PROPERTIES) - .withFieldValue("config_properties", CONFIG_PROPERTIES) - .build(); - IcebergReadSchemaTransform readTransform = - (IcebergReadSchemaTransform) READ_PROVIDER.from(transformConfigRow); + (IcebergReadSchemaTransform) READ_PROVIDER.from(READ_CONFIG_ROW); IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator translator = new IcebergSchemaTransformTranslation.IcebergReadSchemaTransformTranslator(); @@ -178,7 +186,7 @@ public void testReCreateReadTransformFromRow() { IcebergReadSchemaTransform readTransformFromRow = translator.fromConfigRow(row, PipelineOptionsFactory.create()); - assertEquals(transformConfigRow, readTransformFromRow.getConfigurationRow()); + assertEquals(READ_CONFIG_ROW, readTransformFromRow.getConfigurationRow()); } @Test @@ -193,11 +201,9 @@ public void testReadTransformProtoTranslation() properties.put("warehouse", warehouse.location); Row transformConfigRow = - Row.withSchema(READ_PROVIDER.configurationSchema()) + Row.fromRow(READ_CONFIG_ROW) .withFieldValue("table", identifier) - .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", properties) - .withFieldValue("config_properties", CONFIG_PROPERTIES) .build(); IcebergReadSchemaTransform readTransform = @@ -240,4 +246,78 @@ public void testReadTransformProtoTranslation() assertEquals(transformConfigRow, readTransformFromSpec.getConfigurationRow()); } + + @Test + public void testReCreateCdcReadTransformFromRow() { + // setting a subset of fields here. + IcebergCdcReadSchemaTransform readTransform = + (IcebergCdcReadSchemaTransform) READ_CDC_PROVIDER.from(READ_CDC_CONFIG_ROW); + + IcebergCdcReadSchemaTransformTranslator translator = + new IcebergCdcReadSchemaTransformTranslator(); + Row row = translator.toConfigRow(readTransform); + + IcebergCdcReadSchemaTransform readTransformFromRow = + translator.fromConfigRow(row, PipelineOptionsFactory.create()); + + assertEquals(READ_CDC_CONFIG_ROW, readTransformFromRow.getConfigurationRow()); + } + + @Test + public void testCdcReadTransformProtoTranslation() + throws InvalidProtocolBufferException, IOException { + // First build a pipeline + Pipeline p = Pipeline.create(); + String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); + warehouse.createTable(TableIdentifier.parse(identifier), TestFixtures.SCHEMA); + + Map properties = new HashMap<>(CATALOG_PROPERTIES); + properties.put("warehouse", warehouse.location); + + Row transformConfigRow = + Row.fromRow(READ_CDC_CONFIG_ROW) + .withFieldValue("table", identifier) + .withFieldValue("catalog_properties", properties) + .build(); + + IcebergCdcReadSchemaTransform readTransform = + (IcebergCdcReadSchemaTransform) READ_CDC_PROVIDER.from(transformConfigRow); + + PCollectionRowTuple.empty(p).apply(readTransform); + + // Then translate the pipeline to a proto and extract IcebergReadSchemaTransform proto + RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p); + List readTransformProto = + pipelineProto.getComponents().getTransformsMap().values().stream() + .filter( + tr -> { + RunnerApi.FunctionSpec spec = tr.getSpec(); + try { + return spec.getUrn().equals(BeamUrns.getUrn(SCHEMA_TRANSFORM)) + && SchemaTransformPayload.parseFrom(spec.getPayload()) + .getIdentifier() + .equals(READ_CDC_PROVIDER.identifier()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toList()); + assertEquals(1, readTransformProto.size()); + RunnerApi.FunctionSpec spec = readTransformProto.get(0).getSpec(); + + // Check that the proto contains correct values + SchemaTransformPayload payload = SchemaTransformPayload.parseFrom(spec.getPayload()); + Schema schemaFromSpec = SchemaTranslation.schemaFromProto(payload.getConfigurationSchema()); + assertEquals(READ_CDC_PROVIDER.configurationSchema(), schemaFromSpec); + Row rowFromSpec = RowCoder.of(schemaFromSpec).decode(payload.getConfigurationRow().newInput()); + assertEquals(transformConfigRow, rowFromSpec); + + // Use the information in the proto to recreate the IcebergReadSchemaTransform + IcebergCdcReadSchemaTransformTranslator translator = + new IcebergCdcReadSchemaTransformTranslator(); + IcebergCdcReadSchemaTransform readTransformFromSpec = + translator.fromConfigRow(rowFromSpec, PipelineOptionsFactory.create()); + + assertEquals(transformConfigRow, readTransformFromSpec.getConfigurationRow()); + } } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java new file mode 100644 index 000000000000..5efe6cd76a72 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java @@ -0,0 +1,241 @@ +/* + * 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.io.iceberg; + +import static org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy.EARLIEST; +import static org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy.LATEST; +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.stream.Collectors; +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.ImmutableMap; +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.Lists; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.parquet.ParquetReader; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; + +/** Test class for {@link ReadUtils}. */ +public class ReadUtilsTest { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule public TestDataWarehouse warehouse = new TestDataWarehouse(TEMPORARY_FOLDER, "default"); + @Rule public TestName testName = new TestName(); + + @Test + public void testCreateReader() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + + Map> data = + ImmutableMap.>builder() + .put("files1s1.parquet", TestFixtures.FILE1SNAPSHOT1) + .put("file2s1.parquet", TestFixtures.FILE2SNAPSHOT1) + .put("file3s1.parquet", TestFixtures.FILE3SNAPSHOT1) + .build(); + + for (Map.Entry> entry : data.entrySet()) { + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords(entry.getKey(), simpleTable.schema(), entry.getValue())) + .commit(); + } + + int numFiles = 0; + try (CloseableIterable iterable = simpleTable.newScan().planTasks()) { + for (CombinedScanTask combinedScanTask : iterable) { + for (FileScanTask fileScanTask : combinedScanTask.tasks()) { + String fileName = Iterables.getLast(Splitter.on("/").split(fileScanTask.file().path())); + List recordsRead = new ArrayList<>(); + try (ParquetReader reader = ReadUtils.createReader(fileScanTask, simpleTable)) { + reader.forEach(recordsRead::add); + } + + assertEquals(data.get(fileName), recordsRead); + numFiles++; + } + } + } + assertEquals(data.size(), numFiles); + } + + @Test + public void testSnapshotsBetween() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + + Map> data = + ImmutableMap.>builder() + .put("files1s1.parquet", TestFixtures.FILE1SNAPSHOT1) + .put("file2s2.parquet", TestFixtures.FILE2SNAPSHOT2) + .put("file3s3.parquet", TestFixtures.FILE3SNAPSHOT3) + .build(); + + for (Map.Entry> entry : data.entrySet()) { + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords(entry.getKey(), simpleTable.schema(), entry.getValue())) + .commit(); + } + + List originalSnapshots = Lists.newArrayList(simpleTable.snapshots()); + List snapshotsBetween = + ReadUtils.snapshotsBetween( + simpleTable, tableId.toString(), null, simpleTable.currentSnapshot().snapshotId()); + + assertEquals("size", originalSnapshots.size(), snapshotsBetween.size()); + assertEquals( + "snapshot id out of order", + originalSnapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList()), + snapshotsBetween.stream().map(SnapshotInfo::getSnapshotId).collect(Collectors.toList())); + assertEquals( + "sequence number out of order", + originalSnapshots.stream().map(Snapshot::sequenceNumber).collect(Collectors.toList()), + snapshotsBetween.stream() + .map(SnapshotInfo::getSequenceNumber) + .collect(Collectors.toList())); + } + + @Test + public void testResolveFromSnapshotExclusive() throws IOException { + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + + // creates 4 snapshots + warehouse.commitData(simpleTable); + List snapshots = + Lists.newArrayList(simpleTable.snapshots()).stream() + .sorted(Comparator.comparingLong(Snapshot::sequenceNumber)) + .collect(Collectors.toList()); + ; + Snapshot earliest = snapshots.get(0); + Snapshot third = snapshots.get(2); + Snapshot latest = snapshots.get(3); + + IcebergScanConfig baseScanConfig = + IcebergScanConfig.builder() + .setCatalogConfig(IcebergCatalogConfig.builder().build()) + .setScanType(IcebergScanConfig.ScanType.TABLE) + .setTableIdentifier(tableId) + .setSchema(IcebergUtils.icebergSchemaToBeamSchema(simpleTable.schema())) + .build(); + IcebergScanConfig streamingScanConfig = baseScanConfig.toBuilder().setStreaming(true).build(); + + // Note that the incremental append scan is configured with an _exclusive_ starting snapshot, + // so we always return the _parent_ of the target snapshot. + List scanConfigCases = + Arrays.asList( + // batch + TestCase.of(baseScanConfig, earliest.parentId(), "default batch read"), + TestCase.of( + baseScanConfig.toBuilder().setFromSnapshotInclusive(third.snapshotId()).build(), + third.parentId(), + "batch with from_snapshot"), + TestCase.of( + baseScanConfig.toBuilder().setFromTimestamp(third.timestampMillis()).build(), + third.parentId(), + "batch with from_timestamp"), + TestCase.of( + baseScanConfig.toBuilder().setStartingStrategy(EARLIEST).build(), + earliest.parentId(), + "batch with starting_strategy=earliest"), + TestCase.of( + baseScanConfig.toBuilder().setStartingStrategy(LATEST).build(), + latest.parentId(), + "batch with starting_strategy=latest"), + // streaming + TestCase.of(streamingScanConfig, latest.parentId(), "default streaming read"), + TestCase.of( + streamingScanConfig + .toBuilder() + .setFromSnapshotInclusive(third.snapshotId()) + .build(), + third.parentId(), + "streaming with from_snapshot"), + TestCase.of( + streamingScanConfig.toBuilder().setFromTimestamp(third.timestampMillis()).build(), + third.parentId(), + "streaming with from_timestamp"), + TestCase.of( + streamingScanConfig.toBuilder().setStartingStrategy(EARLIEST).build(), + earliest.parentId(), + "streaming with starting_strategy=earliest"), + TestCase.of( + streamingScanConfig.toBuilder().setStartingStrategy(LATEST).build(), + latest.parentId(), + "streaming with starting_strategy=latest")); + + List errors = new ArrayList<>(); + for (TestCase testCase : scanConfigCases) { + @Nullable + Long snapshotId = ReadUtils.getFromSnapshotExclusive(simpleTable, testCase.scanConfig); + if (!Objects.equals(testCase.expectedSnapshotId, snapshotId)) { + errors.add( + String.format( + "\t%s: expected %s but got %s", + testCase.description, testCase.expectedSnapshotId, snapshotId)); + } + } + if (!errors.isEmpty()) { + List snapshotIds = + snapshots.stream().map(Snapshot::snapshotId).collect(Collectors.toList()); + throw new RuntimeException( + "Errors for table with snapshots " + + snapshotIds + + ". Test cases that failed:\n" + + String.join("\n", errors)); + } + } + + static class TestCase { + IcebergScanConfig scanConfig; + @Nullable Long expectedSnapshotId; + String description; + + TestCase(IcebergScanConfig scanConfig, @Nullable Long expectedSnapshotId, String description) { + this.scanConfig = scanConfig; + this.expectedSnapshotId = expectedSnapshotId; + this.description = description; + } + + static TestCase of( + IcebergScanConfig scanConfig, @Nullable Long expectedSnapshotId, String description) { + return new TestCase(scanConfig, expectedSnapshotId, description); + } + } +} diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java index d7a6cd34838c..04f3751b9fbb 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestDataWarehouse.java @@ -17,12 +17,16 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.io.iceberg.TestFixtures.createRecord; import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; import java.io.File; import java.io.IOException; +import java.util.Arrays; import java.util.List; +import java.util.Map; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.iceberg.CatalogProperties; @@ -107,6 +111,11 @@ protected void after() { } } + public DataFile writeData(String filename, Schema schema, List> data) + throws IOException { + return writeRecords(filename, schema, Lists.transform(data, d -> createRecord(schema, d))); + } + public DataFile writeRecords(String filename, Schema schema, List records) throws IOException { return writeRecords(filename, schema, PartitionSpec.unpartitioned(), null, records); @@ -170,4 +179,55 @@ public Catalog.TableBuilder buildTable(TableIdentifier tableId, Schema schema) { public Table loadTable(TableIdentifier tableId) { return catalog.loadTable(tableId); } + + public List> commitData(Table simpleTable) throws IOException { + List> data = + Arrays.asList( + TestFixtures.FILE1SNAPSHOT1, + TestFixtures.FILE2SNAPSHOT1, + TestFixtures.FILE3SNAPSHOT1, + TestFixtures.FILE1SNAPSHOT2, + TestFixtures.FILE2SNAPSHOT2, + TestFixtures.FILE3SNAPSHOT2, + TestFixtures.FILE1SNAPSHOT3, + TestFixtures.FILE2SNAPSHOT3, + TestFixtures.FILE3SNAPSHOT3, + TestFixtures.FILE1SNAPSHOT4, + TestFixtures.FILE2SNAPSHOT4, + TestFixtures.FILE3SNAPSHOT4); + + // first snapshot + simpleTable + .newFastAppend() + .appendFile(writeRecords("file1s1.parquet", simpleTable.schema(), data.get(0))) + .appendFile(writeRecords("file2s1.parquet", simpleTable.schema(), data.get(1))) + .appendFile(writeRecords("file3s1.parquet", simpleTable.schema(), data.get(2))) + .commit(); + + // second snapshot + simpleTable + .newFastAppend() + .appendFile(writeRecords("file1s2.parquet", simpleTable.schema(), data.get(3))) + .appendFile(writeRecords("file2s2.parquet", simpleTable.schema(), data.get(4))) + .appendFile(writeRecords("file3s2.parquet", simpleTable.schema(), data.get(5))) + .commit(); + + // third snapshot + simpleTable + .newFastAppend() + .appendFile(writeRecords("file1s3.parquet", simpleTable.schema(), data.get(6))) + .appendFile(writeRecords("file2s3.parquet", simpleTable.schema(), data.get(7))) + .appendFile(writeRecords("file3s3.parquet", simpleTable.schema(), data.get(8))) + .commit(); + + // fourth snapshot + simpleTable + .newFastAppend() + .appendFile(writeRecords("file1s4.parquet", simpleTable.schema(), data.get(9))) + .appendFile(writeRecords("file2s4.parquet", simpleTable.schema(), data.get(10))) + .appendFile(writeRecords("file3s4.parquet", simpleTable.schema(), data.get(11))) + .commit(); + + return data; + } } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java index a3ab3c8b50d4..820cab5ddcd6 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/TestFixtures.java @@ -58,36 +58,52 @@ public class TestFixtures { ImmutableMap.of("id", 6L, "data", "brainy"), ImmutableMap.of("id", 7L, "data", "film"), ImmutableMap.of("id", 8L, "data", "feta")); + public static final List> FILE1SNAPSHOT4_DATA = + ImmutableList.of( + ImmutableMap.of("id", 9L, "data", "clammy"), + ImmutableMap.of("id", 10L, "data", "evacuate"), + ImmutableMap.of("id", 11L, "data", "tissue")); public static final List> FILE2SNAPSHOT1_DATA = ImmutableList.of( - ImmutableMap.of("id", 10L, "data", "clammy"), - ImmutableMap.of("id", 11L, "data", "evacuate"), - ImmutableMap.of("id", 12L, "data", "tissue")); + ImmutableMap.of("id", 12L, "data", "radical"), + ImmutableMap.of("id", 13L, "data", "collocation"), + ImmutableMap.of("id", 14L, "data", "book")); public static final List> FILE2SNAPSHOT2_DATA = ImmutableList.of( - ImmutableMap.of("id", 14L, "data", "radical"), - ImmutableMap.of("id", 15L, "data", "collocation"), - ImmutableMap.of("id", 16L, "data", "book")); + ImmutableMap.of("id", 15L, "data", "cake"), + ImmutableMap.of("id", 16L, "data", "intrinsic"), + ImmutableMap.of("id", 17L, "data", "paper")); public static final List> FILE2SNAPSHOT3_DATA = ImmutableList.of( - ImmutableMap.of("id", 16L, "data", "cake"), - ImmutableMap.of("id", 17L, "data", "intrinsic"), - ImmutableMap.of("id", 18L, "data", "paper")); + ImmutableMap.of("id", 18L, "data", "ocean"), + ImmutableMap.of("id", 19L, "data", "holistic"), + ImmutableMap.of("id", 20L, "data", "preventative")); + public static final List> FILE2SNAPSHOT4_DATA = + ImmutableList.of( + ImmutableMap.of("id", 21L, "data", "cloud"), + ImmutableMap.of("id", 22L, "data", "zen"), + ImmutableMap.of("id", 23L, "data", "sky")); public static final List> FILE3SNAPSHOT1_DATA = ImmutableList.of( - ImmutableMap.of("id", 20L, "data", "ocean"), - ImmutableMap.of("id", 21L, "data", "holistic"), - ImmutableMap.of("id", 22L, "data", "preventative")); + ImmutableMap.of("id", 24L, "data", "belleview"), + ImmutableMap.of("id", 25L, "data", "overview"), + ImmutableMap.of("id", 26L, "data", "tender")); + public static final List> FILE3SNAPSHOT2_DATA = ImmutableList.of( - ImmutableMap.of("id", 24L, "data", "cloud"), - ImmutableMap.of("id", 25L, "data", "zen"), - ImmutableMap.of("id", 26L, "data", "sky")); + ImmutableMap.of("id", 30L, "data", "ripple"), + ImmutableMap.of("id", 31L, "data", "coral"), + ImmutableMap.of("id", 32L, "data", "tide")); public static final List> FILE3SNAPSHOT3_DATA = ImmutableList.of( - ImmutableMap.of("id", 26L, "data", "belleview"), - ImmutableMap.of("id", 27L, "data", "overview"), - ImmutableMap.of("id", 28L, "data", "tender")); + ImmutableMap.of("id", 33L, "data", "pebble"), + ImmutableMap.of("id", 34L, "data", "lagoon"), + ImmutableMap.of("id", 35L, "data", "warm")); + public static final List> FILE3SNAPSHOT4_DATA = + ImmutableList.of( + ImmutableMap.of("id", 27L, "data", "horizon"), + ImmutableMap.of("id", 28L, "data", "blossom"), + ImmutableMap.of("id", 29L, "data", "meadow")); /* First file in test table */ public static final List FILE1SNAPSHOT1 = @@ -96,6 +112,8 @@ public class TestFixtures { Lists.transform(FILE1SNAPSHOT2_DATA, d -> createRecord(SCHEMA, d)); public static final List FILE1SNAPSHOT3 = Lists.transform(FILE1SNAPSHOT3_DATA, d -> createRecord(SCHEMA, d)); + public static final List FILE1SNAPSHOT4 = + Lists.transform(FILE1SNAPSHOT4_DATA, d -> createRecord(SCHEMA, d)); /* Second file in test table */ public static final List FILE2SNAPSHOT1 = @@ -104,6 +122,8 @@ public class TestFixtures { Lists.transform(FILE2SNAPSHOT2_DATA, d -> createRecord(SCHEMA, d)); public static final List FILE2SNAPSHOT3 = Lists.transform(FILE2SNAPSHOT3_DATA, d -> createRecord(SCHEMA, d)); + public static final List FILE2SNAPSHOT4 = + Lists.transform(FILE2SNAPSHOT4_DATA, d -> createRecord(SCHEMA, d)); /* Third file in test table */ public static final List FILE3SNAPSHOT1 = @@ -112,6 +132,8 @@ public class TestFixtures { Lists.transform(FILE3SNAPSHOT2_DATA, d -> createRecord(SCHEMA, d)); public static final List FILE3SNAPSHOT3 = Lists.transform(FILE3SNAPSHOT3_DATA, d -> createRecord(SCHEMA, d)); + public static final List FILE3SNAPSHOT4 = + Lists.transform(FILE3SNAPSHOT4_DATA, d -> createRecord(SCHEMA, d)); public static final ImmutableList asRows(Iterable records) { ArrayList rows = new ArrayList<>(); diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java index 518470138e90..80019e8b0ac2 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java @@ -17,10 +17,15 @@ */ package org.apache.beam.sdk.io.iceberg.catalog; +import static org.apache.beam.sdk.managed.Managed.ICEBERG; +import static org.apache.beam.sdk.managed.Managed.ICEBERG_CDC; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.sdk.values.PCollection.IsBounded.BOUNDED; +import static org.apache.beam.sdk.values.PCollection.IsBounded.UNBOUNDED; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import com.google.api.services.storage.model.StorageObject; @@ -34,15 +39,18 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.LongStream; import java.util.stream.Stream; +import org.apache.beam.runners.direct.DirectOptions; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; import org.apache.beam.sdk.extensions.gcp.util.GcsUtil; import org.apache.beam.sdk.extensions.gcp.util.gcsfs.GcsPath; import org.apache.beam.sdk.io.iceberg.IcebergUtils; +import org.apache.beam.sdk.io.iceberg.ReadUtils; import org.apache.beam.sdk.managed.Managed; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; @@ -57,6 +65,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.RowFilter; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; @@ -64,6 +73,7 @@ import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableScan; import org.apache.iceberg.catalog.Catalog; @@ -88,6 +98,8 @@ import org.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Instant; +import org.joda.time.LocalDate; +import org.joda.time.LocalTime; import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -147,6 +159,7 @@ public static String warehouse(Class testClass) @Before public void setUp() throws Exception { + OPTIONS.as(DirectOptions.class).setTargetParallelism(3); warehouse = String.format( "%s/%s/%s", @@ -200,7 +213,7 @@ public void cleanUp() throws Exception { private static final String RANDOM = UUID.randomUUID().toString(); @Rule public TestPipeline pipeline = TestPipeline.create(); @Rule public TestName testName = new TestName(); - @Rule public transient Timeout globalTimeout = Timeout.seconds(300); + @Rule public transient Timeout globalTimeout = Timeout.seconds(180); private static final int NUM_SHARDS = 10; private static final Logger LOG = LoggerFactory.getLogger(IcebergCatalogBaseIT.class); private static final Schema DOUBLY_NESTED_ROW_SCHEMA = @@ -235,6 +248,9 @@ public void cleanUp() throws Exception { private static final SimpleFunction ROW_FUNC = new SimpleFunction() { + final long offset2025Millis = + LocalDate.parse("2025-01-01").toDateTime(LocalTime.MIDNIGHT).getMillis(); + @Override public Row apply(Long num) { String strNum = Long.toString(num); @@ -250,6 +266,7 @@ public Row apply(Long num) { .addValue(Float.valueOf(strNum + "." + strNum)) .build(); + long timestampMillis = offset2025Millis + TimeUnit.MICROSECONDS.toHours(num); return Row.withSchema(BEAM_SCHEMA) .addValue("value_" + strNum) .addValue(String.valueOf((char) (97 + num % 5))) @@ -260,8 +277,9 @@ public Row apply(Long num) { .addValue(LongStream.range(0, num % 10).boxed().collect(Collectors.toList())) .addValue(num % 2 == 0 ? null : nestedRow) .addValue(num) - .addValue(new DateTime(num).withZone(DateTimeZone.forOffsetHoursMinutes(3, 25))) - .addValue(DateTimeUtil.timestampFromMicros(num)) + .addValue( + new DateTime(timestampMillis).withZone(DateTimeZone.forOffsetHoursMinutes(3, 25))) + .addValue(DateTimeUtil.timestampFromMicros(timestampMillis * 1000)) .addValue(DateTimeUtil.dateFromDays(Integer.parseInt(strNum))) .addValue(DateTimeUtil.timeFromMicros(num)) .build(); @@ -281,10 +299,16 @@ public Record apply(Row input) { LongStream.range(0, numRecords()).boxed().map(ROW_FUNC::apply).collect(Collectors.toList()); /** Populates the Iceberg table and Returns a {@link List} of expected elements. */ - private List populateTable(Table table) throws IOException { + public List populateTable(Table table) throws IOException { + return populateTable(table, null); + } + + /** Populates the Iceberg table with rows, but overrides one field. */ + private List populateTable(Table table, @Nullable String charOverride) throws IOException { double recordsPerShardFraction = numRecords().doubleValue() / NUM_SHARDS; long maxRecordsPerShard = Math.round(Math.ceil(recordsPerShardFraction)); + table.updateProperties().set("write.metadata.metrics.default", "full").commit(); AppendFiles appendFiles = table.newAppend(); List expectedRows = new ArrayList<>(numRecords()); int totalRecords = 0; @@ -307,6 +331,10 @@ private List populateTable(Table table) throws IOException { ++recordNum, ++totalRecords) { Row expectedBeamRow = ROW_FUNC.apply((long) recordNum); + if (charOverride != null) { + expectedBeamRow = + Row.fromRow(expectedBeamRow).withFieldValue("char", charOverride).build(); + } Record icebergRecord = RECORD_FUNC.apply(expectedBeamRow); writer.write(icebergRecord); @@ -379,19 +407,95 @@ public void testRead() throws Exception { Map config = managedIcebergConfig(tableId()); PCollection rows = - pipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); + pipeline.apply(Managed.read(ICEBERG).withConfig(config)).getSinglePCollection(); + + PAssert.that(rows).containsInAnyOrder(expectedRows); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testStreamingRead() throws Exception { + Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + + List expectedRows = populateTable(table); + + Map config = new HashMap<>(managedIcebergConfig(tableId())); + config.put("streaming", true); + config.put("to_snapshot", table.currentSnapshot().snapshotId()); + + PCollection rows = + pipeline + .apply(Managed.read(ICEBERG_CDC).withConfig(config)) + .getSinglePCollection() + .apply(ReadUtils.extractRecords()); + assertThat(rows.isBounded(), equalTo(UNBOUNDED)); PAssert.that(rows).containsInAnyOrder(expectedRows); pipeline.run().waitUntilFinish(); } + @Test + public void testBatchReadBetweenSnapshots() throws Exception { + runReadBetween(true, false); + } + + @Test + public void testStreamingReadBetweenTimestamps() throws Exception { + runReadBetween(false, true); + } + + @Test + public void testWriteRead() throws IOException { + Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + List expectedRows = populateTable(table); + Map config = managedIcebergConfig(tableId()); + + pipeline + .apply("read", Managed.read(ICEBERG).withConfig(config)) + .getSinglePCollection() + .apply("write", Managed.write(ICEBERG).withConfig(config)); + pipeline.run().waitUntilFinish(); + + List returnedRecords = readRecords(table); + assertThat( + returnedRecords, + containsInAnyOrder(expectedRows.stream().map(RECORD_FUNC::apply).toArray())); + } + + @Test + public void testReadWriteStreaming() throws IOException { + Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + List expectedRows = populateTable(table); + + Map config = managedIcebergConfig(tableId()); + + Map readConfig = new HashMap<>(config); + readConfig.put("to_timestamp", System.currentTimeMillis()); + readConfig.put("streaming", true); + + Map writeConfig = new HashMap<>(config); + writeConfig.put("triggering_frequency_seconds", 5); + + pipeline + .apply("streaming read", Managed.read(ICEBERG_CDC).withConfig(readConfig)) + .getSinglePCollection() + .apply(ReadUtils.extractRecords()) + .apply("streaming write", Managed.write(ICEBERG).withConfig(writeConfig)); + pipeline.run().waitUntilFinish(); + + List returnedRecords = readRecords(table); + assertThat( + returnedRecords, + containsInAnyOrder(expectedRows.stream().map(RECORD_FUNC::apply).toArray())); + } + @Test public void testWrite() throws IOException { // Write with Beam // Expect the sink to create the table Map config = managedIcebergConfig(tableId()); PCollection input = pipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); - input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); + input.apply(Managed.write(ICEBERG).withConfig(config)); pipeline.run().waitUntilFinish(); Table table = catalog.loadTable(TableIdentifier.parse(tableId())); @@ -419,7 +523,7 @@ public void testWriteToPartitionedTable() throws IOException { // Write with Beam Map config = managedIcebergConfig(tableId()); PCollection input = pipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); - input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); + input.apply(Managed.write(ICEBERG).withConfig(config)); pipeline.run().waitUntilFinish(); // Read back and check records are correct @@ -454,9 +558,9 @@ public void testStreamingWrite() throws IOException { .via(instant -> ROW_FUNC.apply(instant.getMillis() % numRecords))) .setRowSchema(BEAM_SCHEMA); - assertThat(input.isBounded(), equalTo(PCollection.IsBounded.UNBOUNDED)); + assertThat(input.isBounded(), equalTo(UNBOUNDED)); - input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); + input.apply(Managed.write(ICEBERG).withConfig(config)); pipeline.run().waitUntilFinish(); List returnedRecords = readRecords(table); @@ -487,9 +591,9 @@ public void testStreamingWriteWithPriorWindowing() throws IOException { .via(instant -> ROW_FUNC.apply(instant.getMillis() % numRecords))) .setRowSchema(BEAM_SCHEMA); - assertThat(input.isBounded(), equalTo(PCollection.IsBounded.UNBOUNDED)); + assertThat(input.isBounded(), equalTo(UNBOUNDED)); - input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); + input.apply(Managed.write(ICEBERG).withConfig(config)); pipeline.run().waitUntilFinish(); List returnedRecords = readRecords(table); @@ -569,7 +673,7 @@ private void writeToDynamicDestinations( input = pipeline.apply(Create.of(inputRows)); } - input.setRowSchema(BEAM_SCHEMA).apply(Managed.write(Managed.ICEBERG).withConfig(writeConfig)); + input.setRowSchema(BEAM_SCHEMA).apply(Managed.write(ICEBERG).withConfig(writeConfig)); pipeline.run().waitUntilFinish(); Table table0 = catalog.loadTable(tableIdentifier0); @@ -631,4 +735,37 @@ public void testStreamToDynamicDestinationsAndKeepFields() throws IOException { public void testStreamToPartitionedDynamicDestinations() throws IOException { writeToDynamicDestinations(null, true, true); } + + public void runReadBetween(boolean useSnapshotBoundary, boolean streaming) throws Exception { + Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + + populateTable(table, "a"); // first snapshot + List expectedRows = populateTable(table, "b"); // second snapshot + Snapshot from = table.currentSnapshot(); + expectedRows.addAll(populateTable(table, "c")); // third snapshot + Snapshot to = table.currentSnapshot(); + populateTable(table, "d"); // fourth snapshot + + Map config = new HashMap<>(managedIcebergConfig(tableId())); + if (useSnapshotBoundary) { + config.put("from_snapshot", from.snapshotId()); + config.put("to_snapshot", to.snapshotId()); + } else { // use timestamp boundary + config.put("from_timestamp", from.timestampMillis() - 1); + config.put("to_timestamp", to.timestampMillis() + 1); + } + config.put("streaming", streaming); + + PCollection rows = + pipeline + .apply(Managed.read(ICEBERG_CDC).withConfig(config)) + .getSinglePCollection() + .apply(ReadUtils.extractRecords()); + + IsBounded expectedBoundedness = streaming ? UNBOUNDED : BOUNDED; + assertEquals(expectedBoundedness, rows.isBounded()); + + PAssert.that(rows).containsInAnyOrder(expectedRows); + pipeline.run().waitUntilFinish(); + } } diff --git a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java index 3372c514191a..abfdb9af5fdd 100644 --- a/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java +++ b/sdks/java/managed/src/main/java/org/apache/beam/sdk/managed/Managed.java @@ -92,6 +92,7 @@ public class Managed { // TODO: Dynamically generate a list of supported transforms public static final String ICEBERG = "iceberg"; + public static final String ICEBERG_CDC = "iceberg_cdc"; public static final String KAFKA = "kafka"; public static final String BIGQUERY = "bigquery"; @@ -99,6 +100,7 @@ public class Managed { public static final Map READ_TRANSFORMS = ImmutableMap.builder() .put(ICEBERG, getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_READ)) + .put(ICEBERG_CDC, getUrn(ExternalTransforms.ManagedTransforms.Urns.ICEBERG_CDC_READ)) .put(KAFKA, getUrn(ExternalTransforms.ManagedTransforms.Urns.KAFKA_READ)) .put(BIGQUERY, getUrn(ExternalTransforms.ManagedTransforms.Urns.BIGQUERY_READ)) .build(); @@ -114,9 +116,14 @@ public class Managed { * supported managed sources are: * *

    - *
  • {@link Managed#ICEBERG} : Read from Apache Iceberg tables - *
  • {@link Managed#KAFKA} : Read from Apache Kafka topics - *
  • {@link Managed#BIGQUERY} : Read from GCP BigQuery tables + *
  • {@link Managed#ICEBERG} : Read from Apache Iceberg tables using IcebergIO + *
  • {@link Managed#ICEBERG_CDC} : CDC Read from Apache Iceberg tables using IcebergIO + *
  • {@link Managed#KAFKA} : Read from Apache Kafka topics using KafkaIO + *
  • {@link Managed#BIGQUERY} : Read from GCP BigQuery tables using BigQueryIO *
*/ public static ManagedTransform read(String source) { @@ -136,9 +143,12 @@ public static ManagedTransform read(String source) { * managed sinks are: * *
    - *
  • {@link Managed#ICEBERG} : Write to Apache Iceberg tables - *
  • {@link Managed#KAFKA} : Write to Apache Kafka topics - *
  • {@link Managed#BIGQUERY} : Write to GCP BigQuery tables + *
  • {@link Managed#ICEBERG} : Write to Apache Iceberg tables IcebergIO + *
  • {@link Managed#KAFKA} : Write to Apache Kafka topics KafkaIO + *
  • {@link Managed#BIGQUERY} : Write to GCP BigQuery tables BigQueryIO *
*/ public static ManagedTransform write(String sink) { diff --git a/sdks/standard_expansion_services.yaml b/sdks/standard_expansion_services.yaml index 623e33fe2e7c..4a5afdfb6c38 100644 --- a/sdks/standard_expansion_services.yaml +++ b/sdks/standard_expansion_services.yaml @@ -51,6 +51,7 @@ # Available through apache_beam.transforms.managed.[Read/Write] - 'beam:schematransform:org.apache.beam:iceberg_write:v1' - 'beam:schematransform:org.apache.beam:iceberg_read:v1' + - 'beam:schematransform:org.apache.beam:iceberg_cdc_read:v1' # TODO(ahmedabu98): Enable this service in a future PR #- gradle_target: 'sdks:java:io:google-cloud-platform:expansion-service:shadowJar'