From bb8751123b4d0580bf5cb3f9c09926f7db211971 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 6 Jan 2025 13:15:32 -0500 Subject: [PATCH 01/32] initial --- .../sdk/io/iceberg/CreateReadTasksDoFn.java | 74 ++++++++++ .../apache/beam/sdk/io/iceberg/IcebergIO.java | 51 +++++-- .../IcebergReadSchemaTransformProvider.java | 102 ++++++++++++-- .../beam/sdk/io/iceberg/IcebergUtils.java | 4 + .../sdk/io/iceberg/IncrementalScanSource.java | 49 +++++++ .../beam/sdk/io/iceberg/ReadFromTasks.java | 112 +++++++++++++++ .../sdk/io/iceberg/ReadTaskDescriptor.java | 56 ++++++++ .../iceberg/SchemaTransformConfiguration.java | 69 --------- .../beam/sdk/io/iceberg/SnapshotRange.java | 55 ++++++++ .../beam/sdk/io/iceberg/TableCache.java | 55 ++++++++ .../sdk/io/iceberg/WatchForSnapshots.java | 132 ++++++++++++++++++ ...cebergReadSchemaTransformProviderTest.java | 5 +- 12 files changed, 673 insertions(+), 91 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/CreateReadTasksDoFn.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTaskDescriptor.java delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SchemaTransformConfiguration.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WatchForSnapshots.java 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..88ae5f364c3f --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/CreateReadTasksDoFn.java @@ -0,0 +1,74 @@ +/* + * 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.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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Creates a collection of {@link FileScanTask}s for each {@link SnapshotRange}. Each task + * represents a data file that was appended within a given snapshot range. + */ +class CreateReadTasksDoFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(CreateReadTasksDoFn.class); + private final Counter numFileScanTasks = + Metrics.counter(CreateReadTasksDoFn.class, "numFileScanTasks"); + private final IcebergCatalogConfig catalogConfig; + + CreateReadTasksDoFn(IcebergCatalogConfig catalogConfig) { + this.catalogConfig = catalogConfig; + } + + @ProcessElement + public void process(@Element SnapshotRange descriptor, OutputReceiver out) + throws IOException, ExecutionException { + Table table = TableCache.get(descriptor.getTableIdentifier(), catalogConfig.catalog()); + + long fromSnapshot = descriptor.getFromSnapshot(); + long toSnapshot = descriptor.getToSnapshot(); + + LOG.info("Planning to scan snapshot range ({}, {}]", fromSnapshot, toSnapshot); + IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); + if (fromSnapshot > -1) { + scan = scan.fromSnapshotExclusive(fromSnapshot); + } + try (CloseableIterable tasks = scan.planFiles()) { + for (FileScanTask task : tasks) { + ReadTaskDescriptor taskDescriptor = + ReadTaskDescriptor.builder() + .setTableIdentifierString(descriptor.getTable()) + .setFileScanTaskJson(ScanTaskParser.toJson(task)) + .setRecordCount(task.file().recordCount()) + .build(); + + numFileScanTasks.inc(); + out.output(taskDescriptor); + } + } + } +} 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..7b239a8581bc 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 @@ -405,6 +405,12 @@ public abstract static class ReadRows extends PTransform expand(PBegin input) { TableIdentifier tableId = @@ -427,15 +451,24 @@ 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())) + .setFromSnapshotExclusive(getFromSnapshotExclusive()) + .setToSnapshot(getToSnapshot()) + .build(); + + @Nullable Duration triggeringFrequency = getTriggeringFrequency(); + if (triggeringFrequency != null) { + return input + .apply(new IncrementalScanSource(scanConfig, triggeringFrequency)) + .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); + } + + return input.apply(Read.from(new ScanSource(scanConfig))); } } } 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..6221efe3c332 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,8 @@ 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; +import org.joda.time.Duration; /** * SchemaTransform implementation for {@link IcebergIO#readRows}. Reads records from Iceberg and @@ -40,11 +48,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 +67,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 +78,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(); @@ -81,14 +89,86 @@ Row getConfigurationRow() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - PCollection output = - input - .getPipeline() - .apply( - IcebergIO.readRows(configuration.getIcebergCatalog()) - .from(TableIdentifier.parse(configuration.getTable()))); + IcebergIO.ReadRows readRows = + IcebergIO.readRows(configuration.getIcebergCatalog()) + .from(TableIdentifier.parse(configuration.getTable())); + + @Nullable Integer triggeringFrequencySeconds = configuration.getTriggeringFrequencySeconds(); + if (triggeringFrequencySeconds != null) { + readRows = + readRows + .fromSnapshotExclusive(configuration.getFromSnapshotExclusive()) + .toSnapshot(configuration.getToSnapshot()) + .withTriggeringFrequency(Duration.standardSeconds(triggeringFrequencySeconds)); + } + + 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_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(); + + @SchemaFieldDescription( + "The frequency at which to poll for new snapshots. An unbounded source is used when this is set.") + abstract @Nullable Integer getTriggeringFrequencySeconds(); + + @SchemaFieldDescription( + "Starts reading from this snapshot ID (exclusive). If unset, the source will " + + "start reading from the oldest snapshot (inclusive).") + abstract @Nullable Long getFromSnapshotExclusive(); + + @SchemaFieldDescription( + "Reads up to this snapshot ID (inclusive). If unset, the source will poll " + + "for new snapshots forever.") + abstract @Nullable Long getToSnapshot(); + + @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 setTriggeringFrequencySeconds(Integer snapshot); + + abstract Builder setFromSnapshotExclusive(Long snapshot); + + abstract Builder setToSnapshot(Long snapshot); + + 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/IcebergUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java index ef19a5881366..2ee70d23eafc 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java @@ -403,6 +403,10 @@ private static Object getIcebergTimestampValue(Object beamValue, boolean shouldA } } + public static Row icebergRecordToBeamRow(org.apache.iceberg.Schema schema, Record record) { + return icebergRecordToBeamRow(icebergSchemaToBeamSchema(schema), record); + } + /** Converts an Iceberg {@link Record} to a Beam {@link Row}. */ public static Row icebergRecordToBeamRow(Schema schema, Record record) { Row.Builder rowBuilder = Row.withSchema(schema); 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..09b424dcbbc3 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java @@ -0,0 +1,49 @@ +/* + * 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 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.joda.time.Duration; + +/** + * An unbounded source that polls for new {@link org.apache.iceberg.Snapshot}s and performs {@link + * org.apache.iceberg.IncrementalAppendScan}s to create a list of {@link + * org.apache.iceberg.FileScanTask}s for each range of snapshots. An SDF is used to process each + * file and output its rows. + */ +class IncrementalScanSource extends PTransform> { + private final Duration pollInterval; + private final IcebergScanConfig scanConfig; + + IncrementalScanSource(IcebergScanConfig scanConfig, Duration pollInterval) { + this.scanConfig = scanConfig; + this.pollInterval = pollInterval; + } + + @Override + public PCollection expand(PBegin input) { + return input + .apply(new WatchForSnapshots(scanConfig, pollInterval)) + .apply(ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) + .apply(ParDo.of(new ReadFromTasks(scanConfig.getCatalogConfig()))); + } +} 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..a1eab61f6cbb --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.concurrent.ExecutionException; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.values.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.parquet.Parquet; + +/** + * Reads Iceberg {@link Record}s from a {@link FileScanTask} and converts to Beam {@link Row}s + * before outputting. + */ +@DoFn.BoundedPerElement +class ReadFromTasks extends DoFn { + private final IcebergCatalogConfig catalogConfig; + + ReadFromTasks(IcebergCatalogConfig catalogConfig) { + this.catalogConfig = catalogConfig; + } + + @ProcessElement + public void process( + @Element ReadTaskDescriptor taskDescriptor, + RestrictionTracker tracker, + OutputReceiver out) + throws IOException, ExecutionException { + Table table = + TableCache.get(taskDescriptor.getTableIdentifierString(), catalogConfig.catalog()); + + FileScanTask task = taskDescriptor.getFileScanTask(); + DataFile dataFile = task.file(); + String filePath = dataFile.path().toString(); + ByteBuffer encryptionKeyMetadata = dataFile.keyMetadata(); + Schema tableSchema = table.schema(); + + // TODO(ahmedabu98): maybe cache this file ref? + InputFile inputFile; + try (FileIO io = table.io()) { + inputFile = io.newInputFile(filePath); + } + if (encryptionKeyMetadata != null) { + inputFile = + table + .encryption() + .decrypt(EncryptedFiles.encryptedInput(inputFile, encryptionKeyMetadata)); + } + + Parquet.ReadBuilder readBuilder = + Parquet.read(inputFile) + .split(task.start(), task.length()) + .project(tableSchema) + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(tableSchema, fileSchema)) + .filter(task.residual()); + + long fromRecord = tracker.currentRestriction().getFrom(); + long toRecord = tracker.currentRestriction().getTo(); + try (CloseableIterable iterable = readBuilder.build()) { + CloseableIterator reader = iterable.iterator(); + // Skip until fromRecord + // TODO(ahmedabu98): this is extremely inefficient + for (long skipped = 0; skipped < fromRecord && reader.hasNext(); ++skipped) { + reader.next(); + } + + for (long l = fromRecord; l < toRecord && reader.hasNext(); ++l) { + if (!tracker.tryClaim(l)) { + break; + } + Record record = reader.next(); + Row row = IcebergUtils.icebergRecordToBeamRow(tableSchema, record); + out.output(row); + } + } + } + + @GetInitialRestriction + public OffsetRange getInitialRange(@Element ReadTaskDescriptor task) { + return new OffsetRange(0, task.getRecordCount()); + } +} 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..2e3f8b481b79 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTaskDescriptor.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import com.google.auto.value.AutoValue; +import java.io.Serializable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +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; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +abstract class ReadTaskDescriptor implements Serializable { + static Builder builder() { + return new AutoValue_ReadTaskDescriptor.Builder(); + } + + abstract String getTableIdentifierString(); + + abstract String getFileScanTaskJson(); + + abstract long getRecordCount(); + + @SchemaIgnore + FileScanTask getFileScanTask() { + return ScanTaskParser.fromJson(getFileScanTaskJson(), true); + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setTableIdentifierString(String table); + + abstract Builder setFileScanTaskJson(String fromSnapshot); + + abstract Builder setRecordCount(long recordCount); + + abstract ReadTaskDescriptor build(); + } +} 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/SnapshotRange.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java new file mode 100644 index 000000000000..81373dc1a2ef --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java @@ -0,0 +1,55 @@ +/* + * 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.io.Serializable; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaIgnore; +import org.apache.iceberg.catalog.TableIdentifier; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +abstract class SnapshotRange implements Serializable { + public static Builder builder() { + return new AutoValue_SnapshotRange.Builder(); + } + + abstract String getTable(); + + abstract long getFromSnapshot(); + + abstract long getToSnapshot(); + + @SchemaIgnore + public TableIdentifier getTableIdentifier() { + return TableIdentifier.parse(getTable()); + } + + @AutoValue.Builder + public abstract static class Builder { + public abstract Builder setTable(String table); + + public abstract Builder setFromSnapshot(Long fromSnapshot); + + public abstract Builder setToSnapshot(Long toSnapshot); + + public abstract SnapshotRange build(); + } +} 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..4c8e29c67fe3 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/TableCache.java @@ -0,0 +1,55 @@ +/* + * 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) throws ExecutionException { + return CACHE.get(identifier, () -> catalog.loadTable(identifier)); + } + + static Table get(String identifier, Catalog catalog) throws ExecutionException { + return get(TableIdentifier.parse(identifier), catalog); + } + + static Table getRefreshed(TableIdentifier identifier, Catalog catalog) throws ExecutionException { + @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) throws ExecutionException { + 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..44835dd6b5c6 --- /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.Collections; +import java.util.List; +import org.apache.beam.sdk.metrics.Gauge; +import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.transforms.Impulse; +import org.apache.beam.sdk.transforms.MapElements; +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.sdk.values.TypeDescriptor; +import org.apache.iceberg.Table; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * Watches for Iceberg table {@link org.apache.iceberg.Snapshot}s and continuously outputs a range + * of snapshots. + * + *

Downstream, a collection of scan tasks are created for each range. + */ +class WatchForSnapshots extends PTransform> { + 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(Impulse.create()) + .apply( + "Watch for snapshots", + Watch.growthOf(new SnapshotPollFn(scanConfig)).withPollInterval(pollInterval)) + .apply( + "Create snapshot intervals", + MapElements.into(TypeDescriptor.of(SnapshotRange.class)) + .via( + result -> { + long from = result.getValue().getKey(); + long to = result.getValue().getValue(); + + return SnapshotRange.builder() + .setTable(scanConfig.getTableIdentifier()) + .setFromSnapshot(from) + .setToSnapshot(to) + .build(); + })); + } + + private static class SnapshotPollFn extends Watch.Growth.PollFn> { + private final Gauge latestSnapshot = Metrics.gauge(SnapshotPollFn.class, "latestSnapshot"); + private final IcebergScanConfig scanConfig; + private Long fromSnapshot; + + SnapshotPollFn(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + this.fromSnapshot = + scanConfig.getFromSnapshotExclusive() != null + ? scanConfig.getFromSnapshotExclusive() + : -1; + } + + @Override + public PollResult> apply(byte[] element, Context c) throws Exception { + // fetch a fresh table to catch updated snapshots + Table table = + TableCache.getRefreshed( + scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); + Instant timestamp = Instant.now(); + + Long currentSnapshot = table.currentSnapshot().snapshotId(); + if (currentSnapshot.equals(fromSnapshot)) { + // no new values since last poll. return empty result. + return getPollResult(null, timestamp); + } + + // we are reading data either up to a specified snapshot or up to the latest available + // snapshot + Long toSnapshot = + scanConfig.getToSnapshot() != null ? scanConfig.getToSnapshot() : currentSnapshot; + latestSnapshot.set(toSnapshot); + + KV fromTo = KV.of(fromSnapshot, toSnapshot); + + // update lower bound to current snapshot + fromSnapshot = currentSnapshot; + + return getPollResult(fromTo, timestamp); + } + + /** Returns an appropriate PollResult based on the requested boundedness. */ + private PollResult> getPollResult( + @Nullable KV fromTo, Instant timestamp) { + List>> timestampedValues = + fromTo == null + ? Collections.emptyList() + : Collections.singletonList(TimestampedValue.of(fromTo, timestamp)); + + return scanConfig.getToSnapshot() != null + ? PollResult.complete(timestampedValues) // stop at specified + : PollResult.incomplete(timestampedValues); // continue forever + } + } +} 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..c62899a341a0 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; @@ -101,8 +102,8 @@ public void testSimpleScan() throws Exception { properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", warehouse.location); - SchemaTransformConfiguration readConfig = - SchemaTransformConfiguration.builder() + Configuration readConfig = + Configuration.builder() .setTable(identifier) .setCatalogName("name") .setCatalogProperties(properties) From 853de4dc32c91787cc2483340d5102c3d815f4e3 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 7 Jan 2025 15:50:03 -0500 Subject: [PATCH 02/32] let CombinedScanTask do splitting (based on Parquet row groups) --- .../sdk/io/iceberg/CreateReadTasksDoFn.java | 12 +-- .../beam/sdk/io/iceberg/IcebergUtils.java | 23 +++++ .../beam/sdk/io/iceberg/ReadFromTasks.java | 95 ++++++++++--------- .../sdk/io/iceberg/ReadTaskDescriptor.java | 39 ++++++-- .../beam/sdk/io/iceberg/ScanTaskReader.java | 25 +---- 5 files changed, 115 insertions(+), 79 deletions(-) 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 index 88ae5f364c3f..e7a64dec06de 100644 --- 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 @@ -22,9 +22,9 @@ import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.iceberg.CombinedScanTask; 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.slf4j.Logger; @@ -57,16 +57,14 @@ public void process(@Element SnapshotRange descriptor, OutputReceiver -1) { scan = scan.fromSnapshotExclusive(fromSnapshot); } - try (CloseableIterable tasks = scan.planFiles()) { - for (FileScanTask task : tasks) { + try (CloseableIterable combinedScanTasks = scan.planTasks()) { + for (CombinedScanTask combinedScanTask : combinedScanTasks) { ReadTaskDescriptor taskDescriptor = ReadTaskDescriptor.builder() .setTableIdentifierString(descriptor.getTable()) - .setFileScanTaskJson(ScanTaskParser.toJson(task)) - .setRecordCount(task.file().recordCount()) + .setCombinedScanTask(combinedScanTask) .build(); - - numFileScanTasks.inc(); + numFileScanTasks.inc(combinedScanTask.filesCount()); out.output(taskDescriptor); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java index 2ee70d23eafc..d250d86959f5 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java @@ -26,21 +26,28 @@ import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.UUID; +import java.util.function.BiFunction; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.PartitionUtil; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.joda.time.Instant; @@ -510,4 +517,20 @@ private static Object getLogicalTypeValue(Object icebergValue, Schema.FieldType // LocalDateTime, LocalDate, LocalTime return icebergValue; } + + 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(); + } + } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index a1eab61f6cbb..d16f6aa4f4b4 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -18,24 +18,27 @@ package org.apache.beam.sdk.io.iceberg; import java.io.IOException; -import java.nio.ByteBuffer; +import java.util.Map; import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.values.Row; -import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.Schema; 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.InputFilesDecryptor; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; 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.Parquet; +import org.checkerframework.checker.nullness.qual.Nullable; /** * Reads Iceberg {@link Record}s from a {@link FileScanTask} and converts to Beam {@link Row}s @@ -57,56 +60,62 @@ public void process( throws IOException, ExecutionException { Table table = TableCache.get(taskDescriptor.getTableIdentifierString(), catalogConfig.catalog()); + @Nullable String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping mapping = + nameMapping != null ? NameMappingParser.fromJson(nameMapping) : NameMapping.empty(); - FileScanTask task = taskDescriptor.getFileScanTask(); - DataFile dataFile = task.file(); - String filePath = dataFile.path().toString(); - ByteBuffer encryptionKeyMetadata = dataFile.keyMetadata(); - Schema tableSchema = table.schema(); - - // TODO(ahmedabu98): maybe cache this file ref? - InputFile inputFile; + InputFilesDecryptor decryptor; try (FileIO io = table.io()) { - inputFile = io.newInputFile(filePath); - } - if (encryptionKeyMetadata != null) { - inputFile = - table - .encryption() - .decrypt(EncryptedFiles.encryptedInput(inputFile, encryptionKeyMetadata)); + decryptor = + new InputFilesDecryptor(taskDescriptor.getCombinedScanTask(), io, table.encryption()); } - Parquet.ReadBuilder readBuilder = - Parquet.read(inputFile) - .split(task.start(), task.length()) - .project(tableSchema) - .createReaderFunc( - fileSchema -> GenericParquetReaders.buildReader(tableSchema, fileSchema)) - .filter(task.residual()); - - long fromRecord = tracker.currentRestriction().getFrom(); - long toRecord = tracker.currentRestriction().getTo(); - try (CloseableIterable iterable = readBuilder.build()) { - CloseableIterator reader = iterable.iterator(); - // Skip until fromRecord - // TODO(ahmedabu98): this is extremely inefficient - for (long skipped = 0; skipped < fromRecord && reader.hasNext(); ++skipped) { - reader.next(); + for (long taskIndex = tracker.currentRestriction().getFrom(); + taskIndex < tracker.currentRestriction().getTo(); + ++taskIndex) { + if (!tracker.tryClaim(taskIndex)) { + return; } + FileScanTask task = taskDescriptor.getFileScanTasks().get((int) taskIndex); + InputFile inputFile = decryptor.getInputFile(task); + Schema tableSchema = table.schema(); + + Map idToConstants = + IcebergUtils.constantsMap( + task, IdentityPartitionConverters::convertConstant, tableSchema); - for (long l = fromRecord; l < toRecord && reader.hasNext(); ++l) { - if (!tracker.tryClaim(l)) { - break; + Parquet.ReadBuilder readBuilder = + Parquet.read(inputFile) + .split(task.start(), task.length()) + .project(tableSchema) + .createReaderFunc( + fileSchema -> + GenericParquetReaders.buildReader(tableSchema, fileSchema, idToConstants)) + .withNameMapping(mapping) + .filter(task.residual()); + + try (CloseableIterable iterable = readBuilder.build()) { + for (Record record : iterable) { + Row row = IcebergUtils.icebergRecordToBeamRow(tableSchema, record); + out.output(row); } - Record record = reader.next(); - Row row = IcebergUtils.icebergRecordToBeamRow(tableSchema, record); - out.output(row); } } } @GetInitialRestriction - public OffsetRange getInitialRange(@Element ReadTaskDescriptor task) { - return new OffsetRange(0, task.getRecordCount()); + public OffsetRange getInitialRange(@Element ReadTaskDescriptor taskDescriptor) { + return new OffsetRange(0, taskDescriptor.getFileScanTaskJsonList().size()); + } + + @GetSize + public double getSize( + @Element ReadTaskDescriptor taskDescriptor, @Restriction OffsetRange restriction) + throws Exception { + double size = 0; + for (long i = restriction.getFrom(); i < restriction.getTo(); i++) { + size += taskDescriptor.getFileScanTasks().get((int) i).sizeBytes(); + } + return size; } } 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 index 2e3f8b481b79..6e27a7b06a51 100644 --- 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 @@ -19,37 +19,64 @@ import com.google.auto.value.AutoValue; import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; import org.apache.beam.sdk.schemas.AutoValueSchema; import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaIgnore; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.ScanTaskParser; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; @DefaultSchema(AutoValueSchema.class) @AutoValue abstract class ReadTaskDescriptor implements Serializable { + private transient @MonotonicNonNull CombinedScanTask cachedCombinedScanTask; + private transient @MonotonicNonNull List cachedFileScanTasks; + static Builder builder() { return new AutoValue_ReadTaskDescriptor.Builder(); } abstract String getTableIdentifierString(); - abstract String getFileScanTaskJson(); + abstract List getFileScanTaskJsonList(); - abstract long getRecordCount(); + @SchemaIgnore + CombinedScanTask getCombinedScanTask() { + if (cachedCombinedScanTask == null) { + cachedCombinedScanTask = new BaseCombinedScanTask(getFileScanTasks()); + } + return cachedCombinedScanTask; + } @SchemaIgnore - FileScanTask getFileScanTask() { - return ScanTaskParser.fromJson(getFileScanTaskJson(), true); + List getFileScanTasks() { + if (cachedFileScanTasks == null) { + cachedFileScanTasks = + getFileScanTaskJsonList().stream() + .map(json -> ScanTaskParser.fromJson(json, true)) + .collect(Collectors.toList()); + } + return cachedFileScanTasks; } @AutoValue.Builder abstract static class Builder { abstract Builder setTableIdentifierString(String table); - abstract Builder setFileScanTaskJson(String fromSnapshot); + abstract Builder setFileScanTaskJsonList(List fromSnapshot); - abstract Builder setRecordCount(long recordCount); + @SchemaIgnore + Builder setCombinedScanTask(CombinedScanTask combinedScanTask) { + List tasks = new ArrayList<>(combinedScanTask.tasks()); + List jsonTasks = + tasks.stream().map(ScanTaskParser::toJson).collect(Collectors.toList()); + return setFileScanTaskJsonList(jsonTasks); + } abstract ReadTaskDescriptor build(); } 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 5784dfd79744..54073d34189e 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.avro.Avro; import org.apache.iceberg.data.IdentityPartitionConverters; @@ -49,9 +44,6 @@ import org.apache.iceberg.io.InputFile; 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; @@ -123,7 +115,8 @@ public boolean advance() throws IOException { DataFile file = fileTask.file(); InputFile input = decryptor.getInputFile(fileTask); Map idToConstants = - constantsMap(fileTask, IdentityPartitionConverters::convertConstant, project); + IcebergUtils.constantsMap( + fileTask, IdentityPartitionConverters::convertConstant, project); CloseableIterable iterable; switch (file.format()) { @@ -171,20 +164,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) { From 69fd98827506840b2729ff1fa44c9f0118316d16 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 7 Jan 2025 16:39:17 -0500 Subject: [PATCH 03/32] perf improv --- .../java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index d16f6aa4f4b4..1dadc17a1a10 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -60,6 +60,9 @@ public void process( throws IOException, ExecutionException { Table table = TableCache.get(taskDescriptor.getTableIdentifierString(), catalogConfig.catalog()); + Schema tableSchema = table.schema(); + org.apache.beam.sdk.schemas.Schema beamSchema = + IcebergUtils.icebergSchemaToBeamSchema(tableSchema); @Nullable String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); NameMapping mapping = nameMapping != null ? NameMappingParser.fromJson(nameMapping) : NameMapping.empty(); @@ -78,7 +81,6 @@ public void process( } FileScanTask task = taskDescriptor.getFileScanTasks().get((int) taskIndex); InputFile inputFile = decryptor.getInputFile(task); - Schema tableSchema = table.schema(); Map idToConstants = IcebergUtils.constantsMap( @@ -96,7 +98,7 @@ public void process( try (CloseableIterable iterable = readBuilder.build()) { for (Record record : iterable) { - Row row = IcebergUtils.icebergRecordToBeamRow(tableSchema, record); + Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); out.output(row); } } From da2f33f619de97e103e00c7215dc0b8217a0e098 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 8 Jan 2025 11:41:52 -0500 Subject: [PATCH 04/32] create one read task descriptor per snapshot range --- .../sdk/io/iceberg/CreateReadTasksDoFn.java | 30 +++++++------- .../sdk/io/iceberg/IcebergCatalogConfig.java | 39 +++++++++++-------- 2 files changed, 40 insertions(+), 29 deletions(-) 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 index e7a64dec06de..e1e89983913e 100644 --- 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 @@ -18,10 +18,13 @@ package org.apache.beam.sdk.io.iceberg; import java.io.IOException; +import java.util.ArrayList; +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.iceberg.BaseCombinedScanTask; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.IncrementalAppendScan; @@ -45,12 +48,12 @@ class CreateReadTasksDoFn extends DoFn { } @ProcessElement - public void process(@Element SnapshotRange descriptor, OutputReceiver out) + public void process(@Element SnapshotRange range, OutputReceiver out) throws IOException, ExecutionException { - Table table = TableCache.get(descriptor.getTableIdentifier(), catalogConfig.catalog()); + Table table = TableCache.get(range.getTableIdentifier(), catalogConfig.catalog()); - long fromSnapshot = descriptor.getFromSnapshot(); - long toSnapshot = descriptor.getToSnapshot(); + long fromSnapshot = range.getFromSnapshot(); + long toSnapshot = range.getToSnapshot(); LOG.info("Planning to scan snapshot range ({}, {}]", fromSnapshot, toSnapshot); IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); @@ -58,15 +61,16 @@ public void process(@Element SnapshotRange descriptor, OutputReceiver combinedScanTasks = scan.planTasks()) { - for (CombinedScanTask combinedScanTask : combinedScanTasks) { - ReadTaskDescriptor taskDescriptor = - ReadTaskDescriptor.builder() - .setTableIdentifierString(descriptor.getTable()) - .setCombinedScanTask(combinedScanTask) - .build(); - numFileScanTasks.inc(combinedScanTask.filesCount()); - out.output(taskDescriptor); - } + List taskList = new ArrayList<>(); + combinedScanTasks.forEach(combined -> taskList.addAll(combined.tasks())); + + ReadTaskDescriptor taskDescriptor = + ReadTaskDescriptor.builder() + .setTableIdentifierString(range.getTable()) + .setCombinedScanTask(new BaseCombinedScanTask(taskList)) + .build(); + numFileScanTasks.inc(taskList.size()); + out.output(taskDescriptor); } } } 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 5307047354b8..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 @@ -24,11 +24,15 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; @AutoValue public abstract class IcebergCatalogConfig implements Serializable { + private transient @MonotonicNonNull Catalog cachedCatalog; + @Pure @Nullable public abstract String getCatalogName(); @@ -47,23 +51,26 @@ public static Builder builder() { } public org.apache.iceberg.catalog.Catalog catalog() { - 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()); + 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); } - return CatalogUtil.buildIcebergCatalog(catalogName, catalogProps, config); + return cachedCatalog; } @AutoValue.Builder From 81ca70998da8ff7146df202f49c01e17abfe662f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 24 Jan 2025 20:27:35 -0500 Subject: [PATCH 05/32] some improvements --- sdks/java/io/iceberg/bqms/build.gradle | 4 +- .../sdk/io/iceberg/CreateReadTasksDoFn.java | 19 +++-- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 8 +-- .../IcebergReadSchemaTransformProvider.java | 9 ++- .../sdk/io/iceberg/IncrementalScanSource.java | 45 +++++++++--- .../beam/sdk/io/iceberg/ReadFromTasks.java | 12 ++-- .../sdk/io/iceberg/ReadTaskDescriptor.java | 15 +++- .../beam/sdk/io/iceberg/SnapshotRange.java | 25 ++++--- .../beam/sdk/io/iceberg/TableCache.java | 15 ++-- .../sdk/io/iceberg/WatchForSnapshots.java | 71 ++++++++----------- 10 files changed, 136 insertions(+), 87 deletions(-) diff --git a/sdks/java/io/iceberg/bqms/build.gradle b/sdks/java/io/iceberg/bqms/build.gradle index e42aafc5f424..3d268b9a8271 100644 --- a/sdks/java/io/iceberg/bqms/build.gradle +++ b/sdks/java/io/iceberg/bqms/build.gradle @@ -21,7 +21,9 @@ plugins { applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.io.iceberg.bqms', - shadowClosure: {}, + shadowClosure: { + relocate "com.google.auth", getJavaRelocatedPath("bqms.com.google.auth") + }, exportJavadoc: false, publish: false, // it's an intermediate jar for io-expansion-service validateShadowJar: false 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 index e1e89983913e..cb6d737b5902 100644 --- 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 @@ -21,6 +21,7 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.DoFn; @@ -30,12 +31,13 @@ import org.apache.iceberg.IncrementalAppendScan; import org.apache.iceberg.Table; import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * Creates a collection of {@link FileScanTask}s for each {@link SnapshotRange}. Each task - * represents a data file that was appended within a given snapshot range. + * represents a portion of a data file that was appended within a given snapshot range. */ class CreateReadTasksDoFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(CreateReadTasksDoFn.class); @@ -51,23 +53,28 @@ class CreateReadTasksDoFn extends DoFn { public void process(@Element SnapshotRange range, OutputReceiver out) throws IOException, ExecutionException { Table table = TableCache.get(range.getTableIdentifier(), catalogConfig.catalog()); - - long fromSnapshot = range.getFromSnapshot(); + @Nullable Long fromSnapshot = range.getFromSnapshot(); long toSnapshot = range.getToSnapshot(); LOG.info("Planning to scan snapshot range ({}, {}]", fromSnapshot, toSnapshot); IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); - if (fromSnapshot > -1) { + if (fromSnapshot != null) { scan = scan.fromSnapshotExclusive(fromSnapshot); } try (CloseableIterable combinedScanTasks = scan.planTasks()) { List taskList = new ArrayList<>(); - combinedScanTasks.forEach(combined -> taskList.addAll(combined.tasks())); + AtomicLong byteSize = new AtomicLong(); + combinedScanTasks.forEach( + combined -> { + taskList.addAll(combined.tasks()); + byteSize.addAndGet(combined.sizeBytes()); + }); ReadTaskDescriptor taskDescriptor = ReadTaskDescriptor.builder() - .setTableIdentifierString(range.getTable()) + .setTableIdentifierString(range.getTableIdentifierString()) .setCombinedScanTask(new BaseCombinedScanTask(taskList)) + .setTotalByteSize(byteSize.get()) .build(); numFileScanTasks.inc(taskList.size()); out.output(taskDescriptor); 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 7b239a8581bc..71b5d3e32a61 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 @@ -460,11 +460,11 @@ public PCollection expand(PBegin input) { .setFromSnapshotExclusive(getFromSnapshotExclusive()) .setToSnapshot(getToSnapshot()) .build(); - - @Nullable Duration triggeringFrequency = getTriggeringFrequency(); - if (triggeringFrequency != null) { + if (getTriggeringFrequency() != null + || scanConfig.getToSnapshot() != null + || scanConfig.getFromSnapshotExclusive() != null) { return input - .apply(new IncrementalScanSource(scanConfig, triggeringFrequency)) + .apply(new IncrementalScanSource(scanConfig, getTriggeringFrequency())) .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); } 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 6221efe3c332..e6da4bccb59a 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 @@ -91,15 +91,14 @@ Row getConfigurationRow() { public PCollectionRowTuple expand(PCollectionRowTuple input) { IcebergIO.ReadRows readRows = IcebergIO.readRows(configuration.getIcebergCatalog()) - .from(TableIdentifier.parse(configuration.getTable())); + .from(TableIdentifier.parse(configuration.getTable())) + .fromSnapshotExclusive(configuration.getFromSnapshotExclusive()) + .toSnapshot(configuration.getToSnapshot()); @Nullable Integer triggeringFrequencySeconds = configuration.getTriggeringFrequencySeconds(); if (triggeringFrequencySeconds != null) { readRows = - readRows - .fromSnapshotExclusive(configuration.getFromSnapshotExclusive()) - .toSnapshot(configuration.getToSnapshot()) - .withTriggeringFrequency(Duration.standardSeconds(triggeringFrequencySeconds)); + readRows.withTriggeringFrequency(Duration.standardSeconds(triggeringFrequencySeconds)); } PCollection output = input.getPipeline().apply(readRows); 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 index 09b424dcbbc3..40bdb1744c61 100644 --- 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 @@ -17,33 +17,58 @@ */ package org.apache.beam.sdk.io.iceberg; +import org.apache.beam.sdk.transforms.Create; 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.iceberg.Table; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; /** - * An unbounded source that polls for new {@link org.apache.iceberg.Snapshot}s and performs {@link - * org.apache.iceberg.IncrementalAppendScan}s to create a list of {@link - * org.apache.iceberg.FileScanTask}s for each range of snapshots. An SDF is used to process each - * file and output its rows. + * An incremental Iceberg source that reads range(s) of table Snapshots. The unbounded + * implementation will continuously poll for new Snapshots at a specified frequency. For each range + * of Snapshots, the transform will create a list of FileScanTasks. An SDF is used to process each + * task and output its rows. */ class IncrementalScanSource extends PTransform> { - private final Duration pollInterval; + private final @Nullable Duration pollInterval; private final IcebergScanConfig scanConfig; - IncrementalScanSource(IcebergScanConfig scanConfig, Duration pollInterval) { + IncrementalScanSource(IcebergScanConfig scanConfig, @Nullable Duration pollInterval) { this.scanConfig = scanConfig; this.pollInterval = pollInterval; } @Override public PCollection expand(PBegin input) { - return input - .apply(new WatchForSnapshots(scanConfig, pollInterval)) - .apply(ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) - .apply(ParDo.of(new ReadFromTasks(scanConfig.getCatalogConfig()))); + PCollection snapshotRanges; + if (pollInterval != null) { // unbounded + snapshotRanges = input.apply(new WatchForSnapshots(scanConfig, pollInterval)); + } else { // bounded + @Nullable Long to = scanConfig.getToSnapshot(); + if (to == null) { + Table table = + TableCache.getRefreshed( + scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); + to = table.currentSnapshot().snapshotId(); + } + snapshotRanges = + input.apply( + "Create Single Snapshot Range", + Create.of( + SnapshotRange.builder() + .setTableIdentifierString(scanConfig.getTableIdentifier()) + .setFromSnapshot(scanConfig.getFromSnapshotExclusive()) + .setToSnapshot(to) + .build())); + } + + return snapshotRanges + .apply( + "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) + .apply("Read Rows From Tasks", ParDo.of(new ReadFromTasks(scanConfig.getCatalogConfig()))); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index 1dadc17a1a10..bf52ce232665 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -41,8 +41,10 @@ import org.checkerframework.checker.nullness.qual.Nullable; /** - * Reads Iceberg {@link Record}s from a {@link FileScanTask} and converts to Beam {@link Row}s - * before outputting. + * An SDF that operates on a collection of {@link FileScanTask}s. For each task, reads Iceberg + * {@link Record}s, and converts to Beam {@link Row}s. + * + *

Can split */ @DoFn.BoundedPerElement class ReadFromTasks extends DoFn { @@ -79,7 +81,7 @@ public void process( if (!tracker.tryClaim(taskIndex)) { return; } - FileScanTask task = taskDescriptor.getFileScanTasks().get((int) taskIndex); + FileScanTask task = taskDescriptor.getFileScanTask(taskIndex); InputFile inputFile = decryptor.getInputFile(task); Map idToConstants = @@ -107,7 +109,7 @@ public void process( @GetInitialRestriction public OffsetRange getInitialRange(@Element ReadTaskDescriptor taskDescriptor) { - return new OffsetRange(0, taskDescriptor.getFileScanTaskJsonList().size()); + return new OffsetRange(0, taskDescriptor.numTasks()); } @GetSize @@ -116,7 +118,7 @@ public double getSize( throws Exception { double size = 0; for (long i = restriction.getFrom(); i < restriction.getTo(); i++) { - size += taskDescriptor.getFileScanTasks().get((int) i).sizeBytes(); + size += taskDescriptor.getFileScanTask(i).sizeBytes(); } return size; } 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 index 6e27a7b06a51..dba2ad019b6b 100644 --- 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 @@ -45,6 +45,12 @@ static Builder builder() { abstract List getFileScanTaskJsonList(); + abstract long getTotalByteSize(); + + int numTasks() { + return getFileScanTaskJsonList().size(); + } + @SchemaIgnore CombinedScanTask getCombinedScanTask() { if (cachedCombinedScanTask == null) { @@ -54,7 +60,7 @@ CombinedScanTask getCombinedScanTask() { } @SchemaIgnore - List getFileScanTasks() { + private List getFileScanTasks() { if (cachedFileScanTasks == null) { cachedFileScanTasks = getFileScanTaskJsonList().stream() @@ -64,12 +70,19 @@ List getFileScanTasks() { return cachedFileScanTasks; } + @SchemaIgnore + FileScanTask getFileScanTask(long index) { + return getFileScanTasks().get((int) index); + } + @AutoValue.Builder abstract static class Builder { abstract Builder setTableIdentifierString(String table); abstract Builder setFileScanTaskJsonList(List fromSnapshot); + abstract Builder setTotalByteSize(long byteSize); + @SchemaIgnore Builder setCombinedScanTask(CombinedScanTask combinedScanTask) { List tasks = new ArrayList<>(combinedScanTask.tasks()); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java index 81373dc1a2ef..edcfef60bb25 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java @@ -23,33 +23,40 @@ import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.apache.beam.sdk.schemas.annotations.SchemaIgnore; import org.apache.iceberg.catalog.TableIdentifier; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.checkerframework.checker.nullness.qual.Nullable; @DefaultSchema(AutoValueSchema.class) @AutoValue abstract class SnapshotRange implements Serializable { - public static Builder builder() { + private transient @MonotonicNonNull TableIdentifier cachedTableIdentifier; + + static Builder builder() { return new AutoValue_SnapshotRange.Builder(); } - abstract String getTable(); + abstract String getTableIdentifierString(); - abstract long getFromSnapshot(); + abstract @Nullable Long getFromSnapshot(); abstract long getToSnapshot(); @SchemaIgnore public TableIdentifier getTableIdentifier() { - return TableIdentifier.parse(getTable()); + if (cachedTableIdentifier == null) { + cachedTableIdentifier = TableIdentifier.parse(getTableIdentifierString()); + } + return cachedTableIdentifier; } @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setTable(String table); + abstract static class Builder { + abstract Builder setTableIdentifierString(String table); - public abstract Builder setFromSnapshot(Long fromSnapshot); + abstract Builder setFromSnapshot(@Nullable Long fromSnapshot); - public abstract Builder setToSnapshot(Long toSnapshot); + abstract Builder setToSnapshot(Long toSnapshot); - public abstract SnapshotRange build(); + abstract SnapshotRange build(); } } 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 index 4c8e29c67fe3..448f3efa805d 100644 --- 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 @@ -31,15 +31,20 @@ class TableCache { private static final Cache CACHE = CacheBuilder.newBuilder().expireAfterWrite(3, TimeUnit.MINUTES).build(); - static Table get(TableIdentifier identifier, Catalog catalog) throws ExecutionException { - return CACHE.get(identifier, () -> catalog.loadTable(identifier)); + 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) throws ExecutionException { + static Table get(String identifier, Catalog catalog) { return get(TableIdentifier.parse(identifier), catalog); } - static Table getRefreshed(TableIdentifier identifier, Catalog catalog) throws ExecutionException { + static Table getRefreshed(TableIdentifier identifier, Catalog catalog) { @Nullable Table table = CACHE.getIfPresent(identifier); if (table == null) { return get(identifier, catalog); @@ -49,7 +54,7 @@ static Table getRefreshed(TableIdentifier identifier, Catalog catalog) throws Ex return table; } - static Table getRefreshed(String identifier, Catalog catalog) throws ExecutionException { + 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 index 44835dd6b5c6..d153ea1e4162 100644 --- 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 @@ -23,7 +23,7 @@ import java.util.List; import org.apache.beam.sdk.metrics.Gauge; import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.transforms.Impulse; +import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Watch; @@ -32,16 +32,17 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.iceberg.Table; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; /** - * Watches for Iceberg table {@link org.apache.iceberg.Snapshot}s and continuously outputs a range - * of snapshots. + * Keeps watches over an Iceberg table continuously outputs a range of snapshots, at the specified + * triggering frequency. * - *

Downstream, a collection of scan tasks are created for each range. + *

A downstream transform will create a list of scan tasks for each range. */ class WatchForSnapshots extends PTransform> { private final Duration pollInterval; @@ -55,77 +56,65 @@ class WatchForSnapshots extends PTransform> { @Override public PCollection expand(PBegin input) { return input - .apply(Impulse.create()) + .apply(Create.of(scanConfig.getTableIdentifier())) .apply( - "Watch for snapshots", + "Watch for Snapshots", Watch.growthOf(new SnapshotPollFn(scanConfig)).withPollInterval(pollInterval)) .apply( - "Create snapshot intervals", - MapElements.into(TypeDescriptor.of(SnapshotRange.class)) - .via( - result -> { - long from = result.getValue().getKey(); - long to = result.getValue().getValue(); - - return SnapshotRange.builder() - .setTable(scanConfig.getTableIdentifier()) - .setFromSnapshot(from) - .setToSnapshot(to) - .build(); - })); + "Strip key", + MapElements.into(TypeDescriptor.of(SnapshotRange.class)).via(KV::getValue)); } - private static class SnapshotPollFn extends Watch.Growth.PollFn> { + private static class SnapshotPollFn extends Watch.Growth.PollFn { private final Gauge latestSnapshot = Metrics.gauge(SnapshotPollFn.class, "latestSnapshot"); private final IcebergScanConfig scanConfig; - private Long fromSnapshot; + private @Nullable Long fromSnapshot; SnapshotPollFn(IcebergScanConfig scanConfig) { this.scanConfig = scanConfig; - this.fromSnapshot = - scanConfig.getFromSnapshotExclusive() != null - ? scanConfig.getFromSnapshotExclusive() - : -1; + this.fromSnapshot = scanConfig.getFromSnapshotExclusive(); } @Override - public PollResult> apply(byte[] element, Context c) throws Exception { + public PollResult apply(String tableIdentifier, Context c) { // fetch a fresh table to catch updated snapshots Table table = - TableCache.getRefreshed( - scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); + TableCache.getRefreshed(tableIdentifier, scanConfig.getCatalogConfig().catalog()); Instant timestamp = Instant.now(); Long currentSnapshot = table.currentSnapshot().snapshotId(); if (currentSnapshot.equals(fromSnapshot)) { - // no new values since last poll. return empty result. + // no new snapshot since last poll. return empty result. return getPollResult(null, timestamp); } - // we are reading data either up to a specified snapshot or up to the latest available - // snapshot - Long toSnapshot = - scanConfig.getToSnapshot() != null ? scanConfig.getToSnapshot() : currentSnapshot; + // if no upper bound is specified, we read up to the current snapshot + Long toSnapshot = MoreObjects.firstNonNull(scanConfig.getSnapshot(), currentSnapshot); latestSnapshot.set(toSnapshot); - KV fromTo = KV.of(fromSnapshot, toSnapshot); + SnapshotRange range = + SnapshotRange.builder() + .setFromSnapshot(fromSnapshot) + .setToSnapshot(toSnapshot) + .setTableIdentifierString(tableIdentifier) + .build(); // update lower bound to current snapshot fromSnapshot = currentSnapshot; - return getPollResult(fromTo, timestamp); + return getPollResult(range, timestamp); } /** Returns an appropriate PollResult based on the requested boundedness. */ - private PollResult> getPollResult( - @Nullable KV fromTo, Instant timestamp) { - List>> timestampedValues = - fromTo == null + private PollResult getPollResult( + @Nullable SnapshotRange range, Instant timestamp) { + List> timestampedValues = + range == null ? Collections.emptyList() - : Collections.singletonList(TimestampedValue.of(fromTo, timestamp)); + : Collections.singletonList(TimestampedValue.of(range, timestamp)); return scanConfig.getToSnapshot() != null - ? PollResult.complete(timestampedValues) // stop at specified + ? PollResult.complete(timestampedValues) // stop at specified snapshot : PollResult.incomplete(timestampedValues); // continue forever } } From e319d76b82e960c14d87f1053e34b5df1bc1febc Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 30 Jan 2025 16:07:51 -0500 Subject: [PATCH 06/32] use GiB for streaming, Redistribute for batch; update docs --- sdks/java/io/iceberg/build.gradle | 1 + .../sdk/io/iceberg/CreateReadTasksDoFn.java | 61 +++++---- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 23 +++- .../IcebergReadSchemaTransformProvider.java | 4 +- .../sdk/io/iceberg/IncrementalScanSource.java | 99 ++++++++++---- .../sdk/io/iceberg/ReadFromGroupedTasks.java | 114 ++++++++++++++++ .../beam/sdk/io/iceberg/ReadFromTasks.java | 128 +++++++++--------- .../apache/beam/sdk/io/iceberg/ReadTask.java | 79 +++++++++++ .../sdk/io/iceberg/ReadTaskDescriptor.java | 71 ++-------- .../beam/sdk/io/iceberg/SnapshotRange.java | 4 +- .../sdk/io/iceberg/WatchForSnapshots.java | 8 +- 11 files changed, 409 insertions(+), 183 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromGroupedTasks.java create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTask.java diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 41e12921e6f8..65e67a7b01b8 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" 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 index cb6d737b5902..d3536b42c7d5 100644 --- 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 @@ -18,30 +18,29 @@ package org.apache.beam.sdk.io.iceberg; import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.beam.sdk.values.KV; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.IncrementalAppendScan; +import org.apache.iceberg.ScanTaskParser; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.io.CloseableIterable; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Creates a collection of {@link FileScanTask}s for each {@link SnapshotRange}. Each task - * represents a portion of a data file that was appended within a given snapshot range. + * Scans the given {@link SnapshotRange}, 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 { +class CreateReadTasksDoFn extends DoFn> { private static final Logger LOG = LoggerFactory.getLogger(CreateReadTasksDoFn.class); - private final Counter numFileScanTasks = + private static final Counter numFileScanTasks = Metrics.counter(CreateReadTasksDoFn.class, "numFileScanTasks"); private final IcebergCatalogConfig catalogConfig; @@ -50,34 +49,42 @@ class CreateReadTasksDoFn extends DoFn { } @ProcessElement - public void process(@Element SnapshotRange range, OutputReceiver out) + public void process( + @Element SnapshotRange range, OutputReceiver> out) throws IOException, ExecutionException { Table table = TableCache.get(range.getTableIdentifier(), catalogConfig.catalog()); - @Nullable Long fromSnapshot = range.getFromSnapshot(); + @Nullable Long fromSnapshot = range.getFromSnapshotExclusive(); long toSnapshot = range.getToSnapshot(); LOG.info("Planning to scan snapshot range ({}, {}]", fromSnapshot, toSnapshot); - IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); + IncrementalAppendScan scan = + table + .newIncrementalAppendScan() + .toSnapshot(toSnapshot) + .option(TableProperties.SPLIT_SIZE, String.valueOf(TableProperties.SPLIT_SIZE_DEFAULT)); if (fromSnapshot != null) { scan = scan.fromSnapshotExclusive(fromSnapshot); } - try (CloseableIterable combinedScanTasks = scan.planTasks()) { - List taskList = new ArrayList<>(); - AtomicLong byteSize = new AtomicLong(); - combinedScanTasks.forEach( - combined -> { - taskList.addAll(combined.tasks()); - byteSize.addAndGet(combined.sizeBytes()); - }); - ReadTaskDescriptor taskDescriptor = - ReadTaskDescriptor.builder() - .setTableIdentifierString(range.getTableIdentifierString()) - .setCombinedScanTask(new BaseCombinedScanTask(taskList)) - .setTotalByteSize(byteSize.get()) - .build(); - numFileScanTasks.inc(taskList.size()); - out.output(taskDescriptor); + try (CloseableIterable combinedScanTasks = scan.planTasks()) { + for (CombinedScanTask combinedScanTask : combinedScanTasks) { + // A single DataFile can be broken up into multiple FileScanTasks + // if it is large enough. + for (FileScanTask fileScanTask : combinedScanTask.tasks()) { + ReadTask task = + ReadTask.builder() + .setTableIdentifierString(range.getTableIdentifierString()) + .setFileScanTaskJson(ScanTaskParser.toJson(fileScanTask)) + .setByteSize(fileScanTask.sizeBytes()) + .build(); + ReadTaskDescriptor descriptor = + ReadTaskDescriptor.builder() + .setTableIdentifierString(range.getTableIdentifierString()) + .build(); + out.output(KV.of(descriptor, task)); + numFileScanTasks.inc(); + } + } } } } 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 71b5d3e32a61..ca8bd64ca07a 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 @@ -81,10 +81,15 @@ * template to use dynamic destinations (see the `Dynamic Destinations` section below for details). * * - * {@code triggering_frequency_seconds} {@code int} Required for streaming writes. Roughly every + * {@code triggering_frequency_seconds} + * {@code int} + * + *

Sink: 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. - * + *

Source: Enables streaming reads. Roughly every {@code triggering_frequency_seconds} duration, the source + * will scan the table for new snapshots and read new records. + * * * * {@code catalog_name} {@code str} The name of the catalog. Defaults to {@code apache-beam-}. @@ -101,6 +106,20 @@ * implementation, but this list * is a good starting point. * + * + * {@code from_snapshot_exclusive} + * {@code long} + * For the source; starts reading from this snapshot ID (exclusive). If unset, it will start reading from the + * oldest snapshot (inclusive). + * + * + * + * {@code to_snapshot} + * {@code long} + * For the source; Reads up to this snapshot ID (inclusive). If unset and the source is bounded, it will read + * up to the current snapshot (inclusive). If unset and source is unbounded, it will continue polling for new snapshots forever. + * + * * * *

Additional configuration options are provided in the `Pre-filtering Options` section below, 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 e6da4bccb59a..e5350ce58ce2 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 @@ -139,8 +139,8 @@ static Builder builder() { abstract @Nullable Long getFromSnapshotExclusive(); @SchemaFieldDescription( - "Reads up to this snapshot ID (inclusive). If unset, the source will poll " - + "for new snapshots forever.") + "Reads up to this snapshot ID (inclusive). If unset and the source is bounded, it will read up to the current snapshot. " + + "The unbounded source will continue polling for new snapshots forever.") abstract @Nullable Long getToSnapshot(); @AutoValue.Builder 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 index 40bdb1744c61..f8a7d6f2c7d4 100644 --- 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 @@ -17,23 +17,36 @@ */ package org.apache.beam.sdk.io.iceberg; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.GroupIntoBatches; 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.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; +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.iceberg.Table; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; /** - * An incremental Iceberg source that reads range(s) of table Snapshots. The unbounded - * implementation will continuously poll for new Snapshots at a specified frequency. For each range - * of Snapshots, the transform will create a list of FileScanTasks. An SDF is used to process each - * task and output its rows. + * An Iceberg source that reads a table incrementally using range(s) of table Snapshots. The + * unbounded implementation will continuously poll for new Snapshots at the specified frequency. A + * collection of FileScanTasks are created for each snapshot range. An SDF (shared by bounded and + * unbounded implementations) is used to process each task and output Beam rows. */ class IncrementalScanSource extends PTransform> { + // For the unbounded implementation. + private static final long MAX_FILES_BATCH_BYTE_SIZE = 1L << 32; // 4 GB private final @Nullable Duration pollInterval; private final IcebergScanConfig scanConfig; @@ -44,31 +57,65 @@ class IncrementalScanSource extends PTransform> { @Override public PCollection expand(PBegin input) { - PCollection snapshotRanges; - if (pollInterval != null) { // unbounded - snapshotRanges = input.apply(new WatchForSnapshots(scanConfig, pollInterval)); - } else { // bounded - @Nullable Long to = scanConfig.getToSnapshot(); - if (to == null) { - Table table = - TableCache.getRefreshed( - scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); - to = table.currentSnapshot().snapshotId(); - } - snapshotRanges = - input.apply( - "Create Single Snapshot Range", - Create.of( - SnapshotRange.builder() - .setTableIdentifierString(scanConfig.getTableIdentifier()) - .setFromSnapshot(scanConfig.getFromSnapshotExclusive()) - .setToSnapshot(to) - .build())); - } + Table table = + TableCache.getRefreshed( + scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); + long to = + MoreObjects.firstNonNull(scanConfig.getToSnapshot(), table.currentSnapshot().snapshotId()); - return snapshotRanges + PCollection rows = + pollInterval == null ? readBounded(input, to) : readUnbounded(input, pollInterval); + return rows.setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); + } + + /** + * Watches for new snapshots and creates tasks for each range. Using GiB for autosharding, this + * groups tasks in batches of up to 4GB, then reads from each batch using an SDF. + */ + private PCollection readUnbounded(PBegin input, Duration duration) { + return input + .apply("Watch for Snapshots", new WatchForSnapshots(scanConfig, duration)) + .apply( + "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) + .setCoder(KvCoder.of(ReadTaskDescriptor.CODER, ReadTask.CODER)) + .apply( + "Apply User Trigger", + Window.>into(new GlobalWindows()) + .triggering( + Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane().plusDelayOf(duration))) + .discardingFiredPanes()) + .apply( + GroupIntoBatches.ofByteSize( + MAX_FILES_BATCH_BYTE_SIZE, ReadTask::getByteSize) + .withMaxBufferingDuration(duration) + .withShardedKey()) + .setCoder( + KvCoder.of( + ShardedKey.Coder.of(ReadTaskDescriptor.CODER), IterableCoder.of(ReadTask.CODER))) + .apply( + "Read Rows From Grouped Tasks", + ParDo.of(new ReadFromGroupedTasks(scanConfig.getCatalogConfig()))); + } + + /** + * Scans a single snapshot range and creates read tasks. Tasks are redistributed and processed + * individually using a regular DoFn. + */ + private PCollection readBounded(PBegin input, long toSnapshot) { + return input + .apply( + "Create Single Snapshot Range", + Create.of( + SnapshotRange.builder() + .setTableIdentifierString(scanConfig.getTableIdentifier()) + .setFromSnapshotExclusive(scanConfig.getFromSnapshotExclusive()) + .setToSnapshot(toSnapshot) + .build())) .apply( "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) + .setCoder(KvCoder.of(ReadTaskDescriptor.CODER, ReadTask.CODER)) + .apply(Redistribute.arbitrarily()) .apply("Read Rows From Tasks", ParDo.of(new ReadFromTasks(scanConfig.getCatalogConfig()))); } } 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..ed5e674ff54f --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromGroupedTasks.java @@ -0,0 +1,114 @@ +/* + * 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.ReadFromTasks.getReader; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutionException; +import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.util.ShardedKey; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.Row; +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.FileScanTask; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +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 split granularity is set to the incoming batch size, i.e. the number of potential splits + * equals the batch size. + */ +@DoFn.BoundedPerElement +class ReadFromGroupedTasks + extends DoFn, Iterable>, Row> { + private final IcebergCatalogConfig catalogConfig; + + ReadFromGroupedTasks(IcebergCatalogConfig catalogConfig) { + this.catalogConfig = catalogConfig; + } + + @ProcessElement + public void process( + @Element KV, Iterable> element, + RestrictionTracker tracker, + OutputReceiver out) + throws IOException, ExecutionException { + String tableIdentifier = element.getKey().getKey().getTableIdentifierString(); + List readTasks = Lists.newArrayList(element.getValue()); + Table table = TableCache.get(tableIdentifier, catalogConfig.catalog()); + Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); + @Nullable String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + NameMapping mapping = + nameMapping != null ? NameMappingParser.fromJson(nameMapping) : NameMapping.empty(); + + // 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; + } + FileScanTask task = readTasks.get((int) taskIndex).getFileScanTask(); + try (CloseableIterable reader = getReader(task, table, mapping)) { + for (Record record : reader) { + Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); + out.output(row); + } + } + } + } + + @GetInitialRestriction + public OffsetRange getInitialRange( + @Element KV, Iterable> element) { + return new OffsetRange(0, Iterables.size(element.getValue())); + } + + @GetSize + public double getSize( + @Element KV, Iterable> element, + @Restriction OffsetRange restriction) + throws Exception { + double size = 0; + Iterator iterator = element.getValue().iterator(); + for (long i = 0; i < restriction.getTo() && iterator.hasNext(); i++) { + ReadTask task = iterator.next(); + if (i >= restriction.getFrom()) { + 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 index bf52ce232665..a8439c11762b 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -18,36 +18,40 @@ package org.apache.beam.sdk.io.iceberg; import java.io.IOException; +import java.util.Collection; import java.util.Map; import java.util.concurrent.ExecutionException; -import org.apache.beam.sdk.io.range.OffsetRange; +import org.apache.beam.sdk.schemas.Schema; 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.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.Schema; 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.InputFilesDecryptor; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedInputFile; +import org.apache.iceberg.hadoop.HadoopInputFile; import org.apache.iceberg.io.CloseableIterable; 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.Parquet; +import org.apache.iceberg.parquet.ParquetReader; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; import org.checkerframework.checker.nullness.qual.Nullable; /** - * An SDF that operates on a collection of {@link FileScanTask}s. For each task, reads Iceberg - * {@link Record}s, and converts to Beam {@link Row}s. + * Bounded read implementation. * - *

Can split + *

For each {@link ReadTask}, reads Iceberg {@link Record}s, and converts to Beam {@link Row}s. */ -@DoFn.BoundedPerElement -class ReadFromTasks extends DoFn { +class ReadFromTasks extends DoFn, Row> { private final IcebergCatalogConfig catalogConfig; ReadFromTasks(IcebergCatalogConfig catalogConfig) { @@ -55,71 +59,69 @@ class ReadFromTasks extends DoFn { } @ProcessElement - public void process( - @Element ReadTaskDescriptor taskDescriptor, - RestrictionTracker tracker, - OutputReceiver out) + public void process(@Element KV element, OutputReceiver out) throws IOException, ExecutionException { - Table table = - TableCache.get(taskDescriptor.getTableIdentifierString(), catalogConfig.catalog()); - Schema tableSchema = table.schema(); - org.apache.beam.sdk.schemas.Schema beamSchema = - IcebergUtils.icebergSchemaToBeamSchema(tableSchema); + String tableIdentifier = element.getKey().getTableIdentifierString(); + ReadTask readTask = element.getValue(); + Table table = TableCache.get(tableIdentifier, catalogConfig.catalog()); + Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); @Nullable String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); NameMapping mapping = nameMapping != null ? NameMappingParser.fromJson(nameMapping) : NameMapping.empty(); - InputFilesDecryptor decryptor; - try (FileIO io = table.io()) { - decryptor = - new InputFilesDecryptor(taskDescriptor.getCombinedScanTask(), io, table.encryption()); - } + FileScanTask task = readTask.getFileScanTask(); - for (long taskIndex = tracker.currentRestriction().getFrom(); - taskIndex < tracker.currentRestriction().getTo(); - ++taskIndex) { - if (!tracker.tryClaim(taskIndex)) { - return; + try (CloseableIterable reader = getReader(task, table, mapping)) { + for (Record record : reader) { + Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); + out.output(row); } - FileScanTask task = taskDescriptor.getFileScanTask(taskIndex); - InputFile inputFile = decryptor.getInputFile(task); + } + } - Map idToConstants = - IcebergUtils.constantsMap( - task, IdentityPartitionConverters::convertConstant, tableSchema); + 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"); - Parquet.ReadBuilder readBuilder = - Parquet.read(inputFile) - .split(task.start(), task.length()) - .project(tableSchema) - .createReaderFunc( - fileSchema -> - GenericParquetReaders.buildReader(tableSchema, fileSchema, idToConstants)) - .withNameMapping(mapping) - .filter(task.residual()); + static ParquetReader getReader(FileScanTask task, Table table, NameMapping mapping) { + 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 = + IcebergUtils.constantsMap( + task, IdentityPartitionConverters::convertConstant, table.schema()); - try (CloseableIterable iterable = readBuilder.build()) { - for (Record record : iterable) { - Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); - out.output(row); - } + 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(1 << 20); // 1MB - @GetInitialRestriction - public OffsetRange getInitialRange(@Element ReadTaskDescriptor taskDescriptor) { - return new OffsetRange(0, taskDescriptor.numTasks()); - } - - @GetSize - public double getSize( - @Element ReadTaskDescriptor taskDescriptor, @Restriction OffsetRange restriction) - throws Exception { - double size = 0; - for (long i = restriction.getFrom(); i < restriction.getTo(); i++) { - size += taskDescriptor.getFileScanTask(i).sizeBytes(); - } - return size; + return new ParquetReader<>( + inputFile, + table.schema(), + optionsBuilder.build(), + fileSchema -> GenericParquetReaders.buildReader(table.schema(), fileSchema, idToConstants), + mapping, + task.residual(), + false, + true); } } 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..964dde36608f --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadTask.java @@ -0,0 +1,79 @@ +/* + * 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.coders.Coder; +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.SchemaIgnore; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ScanTaskParser; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; + +@DefaultSchema(AutoValueSchema.class) +@AutoValue +abstract class ReadTask { + static final Coder CODER; + + static { + try { + CODER = SchemaRegistry.createDefault().getSchemaCoder(ReadTask.class); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } + + private transient @MonotonicNonNull FileScanTask cachedFileScanTask; + + static Builder builder() { + return new AutoValue_ReadTask.Builder(); + } + + abstract String getTableIdentifierString(); + + abstract String getFileScanTaskJson(); + + abstract long getByteSize(); + + @SchemaIgnore + FileScanTask getFileScanTask() { + if (cachedFileScanTask == null) { + cachedFileScanTask = ScanTaskParser.fromJson(getFileScanTaskJson(), true); + } + return cachedFileScanTask; + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setTableIdentifierString(String table); + + abstract Builder setFileScanTaskJson(String jsonTask); + + abstract Builder setByteSize(long size); + + @SchemaIgnore + Builder setFileScanTask(FileScanTask task) { + return setFileScanTaskJson(ScanTaskParser.toJson(task)); + } + + 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 index dba2ad019b6b..4474a893c215 100644 --- 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 @@ -18,24 +18,25 @@ package org.apache.beam.sdk.io.iceberg; import com.google.auto.value.AutoValue; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.stream.Collectors; +import org.apache.beam.sdk.coders.Coder; 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.SchemaIgnore; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.ScanTaskParser; -import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +/** Describes the table and snapshot a {@link ReadTask} belongs to. */ @DefaultSchema(AutoValueSchema.class) @AutoValue -abstract class ReadTaskDescriptor implements Serializable { - private transient @MonotonicNonNull CombinedScanTask cachedCombinedScanTask; - private transient @MonotonicNonNull List cachedFileScanTasks; +abstract class ReadTaskDescriptor { + static final Coder CODER; + + static { + try { + CODER = SchemaRegistry.createDefault().getSchemaCoder(ReadTaskDescriptor.class); + } catch (NoSuchSchemaException e) { + throw new RuntimeException(e); + } + } static Builder builder() { return new AutoValue_ReadTaskDescriptor.Builder(); @@ -43,54 +44,10 @@ static Builder builder() { abstract String getTableIdentifierString(); - abstract List getFileScanTaskJsonList(); - - abstract long getTotalByteSize(); - - int numTasks() { - return getFileScanTaskJsonList().size(); - } - - @SchemaIgnore - CombinedScanTask getCombinedScanTask() { - if (cachedCombinedScanTask == null) { - cachedCombinedScanTask = new BaseCombinedScanTask(getFileScanTasks()); - } - return cachedCombinedScanTask; - } - - @SchemaIgnore - private List getFileScanTasks() { - if (cachedFileScanTasks == null) { - cachedFileScanTasks = - getFileScanTaskJsonList().stream() - .map(json -> ScanTaskParser.fromJson(json, true)) - .collect(Collectors.toList()); - } - return cachedFileScanTasks; - } - - @SchemaIgnore - FileScanTask getFileScanTask(long index) { - return getFileScanTasks().get((int) index); - } - @AutoValue.Builder abstract static class Builder { abstract Builder setTableIdentifierString(String table); - abstract Builder setFileScanTaskJsonList(List fromSnapshot); - - abstract Builder setTotalByteSize(long byteSize); - - @SchemaIgnore - Builder setCombinedScanTask(CombinedScanTask combinedScanTask) { - List tasks = new ArrayList<>(combinedScanTask.tasks()); - List jsonTasks = - tasks.stream().map(ScanTaskParser::toJson).collect(Collectors.toList()); - return setFileScanTaskJsonList(jsonTasks); - } - abstract ReadTaskDescriptor build(); } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java index edcfef60bb25..9ba0c04d22ad 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java @@ -37,7 +37,7 @@ static Builder builder() { abstract String getTableIdentifierString(); - abstract @Nullable Long getFromSnapshot(); + abstract @Nullable Long getFromSnapshotExclusive(); abstract long getToSnapshot(); @@ -53,7 +53,7 @@ public TableIdentifier getTableIdentifier() { abstract static class Builder { abstract Builder setTableIdentifierString(String table); - abstract Builder setFromSnapshot(@Nullable Long fromSnapshot); + abstract Builder setFromSnapshotExclusive(@Nullable Long fromSnapshot); abstract Builder setToSnapshot(Long toSnapshot); 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 index d153ea1e4162..75f5adbf8656 100644 --- 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 @@ -39,10 +39,10 @@ import org.joda.time.Instant; /** - * Keeps watches over an Iceberg table continuously outputs a range of snapshots, at the specified - * triggering frequency. + * 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 scan tasks for each range. + *

A downstream transform will create a list of read tasks for each range. */ class WatchForSnapshots extends PTransform> { private final Duration pollInterval; @@ -94,7 +94,7 @@ public PollResult apply(String tableIdentifier, Context c) { SnapshotRange range = SnapshotRange.builder() - .setFromSnapshot(fromSnapshot) + .setFromSnapshotExclusive(fromSnapshot) .setToSnapshot(toSnapshot) .setTableIdentifierString(tableIdentifier) .build(); From af1ec856c623e33fad8d30d9fbf496e32649a77b Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 3 Feb 2025 10:47:31 -0500 Subject: [PATCH 07/32] use static value --- .../java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index a8439c11762b..be9c0750ac57 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -52,6 +52,8 @@ *

For each {@link ReadTask}, reads Iceberg {@link Record}s, and converts to Beam {@link Row}s. */ class ReadFromTasks extends DoFn, Row> { + // default is 8MB. keep this low to avoid overwhelming memory + private static final int MAX_FILE_BUFFER_SIZE = 1 << 20; // 1MB private final IcebergCatalogConfig catalogConfig; ReadFromTasks(IcebergCatalogConfig catalogConfig) { @@ -112,7 +114,7 @@ static ParquetReader getReader(FileScanTask task, Table table, NameMappi optionsBuilder = optionsBuilder .withRange(task.start(), task.start() + task.length()) - .withMaxAllocationInBytes(1 << 20); // 1MB + .withMaxAllocationInBytes(MAX_FILE_BUFFER_SIZE); return new ParquetReader<>( inputFile, From f5d32685a74cd50872f4fbe36918c447539d83f4 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 3 Feb 2025 16:27:33 -0500 Subject: [PATCH 08/32] add some test --- .../beam/sdk/io/iceberg/IcebergUtils.java | 27 --- .../sdk/io/iceberg/IncrementalScanSource.java | 18 +- .../sdk/io/iceberg/ReadFromGroupedTasks.java | 4 +- .../beam/sdk/io/iceberg/ReadFromTasks.java | 64 +----- .../apache/beam/sdk/io/iceberg/ReadUtils.java | 109 ++++++++++ .../beam/sdk/io/iceberg/ScanTaskReader.java | 3 +- .../sdk/io/iceberg/WatchForSnapshots.java | 17 +- .../sdk/io/iceberg/IcebergIOReadTest.java | 192 +++++++++++++++--- ...cebergReadSchemaTransformProviderTest.java | 186 ++++++++++++----- .../beam/sdk/io/iceberg/ReadUtilsTest.java | 85 ++++++++ .../iceberg/catalog/IcebergCatalogBaseIT.java | 47 +++++ 11 files changed, 564 insertions(+), 188 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java index d250d86959f5..ef19a5881366 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergUtils.java @@ -26,28 +26,21 @@ import java.time.OffsetDateTime; import java.time.ZoneOffset; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.Set; import java.util.UUID; -import java.util.function.BiFunction; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.Row; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.PartitionUtil; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.DateTime; import org.joda.time.Instant; @@ -410,10 +403,6 @@ private static Object getIcebergTimestampValue(Object beamValue, boolean shouldA } } - public static Row icebergRecordToBeamRow(org.apache.iceberg.Schema schema, Record record) { - return icebergRecordToBeamRow(icebergSchemaToBeamSchema(schema), record); - } - /** Converts an Iceberg {@link Record} to a Beam {@link Row}. */ public static Row icebergRecordToBeamRow(Schema schema, Record record) { Row.Builder rowBuilder = Row.withSchema(schema); @@ -517,20 +506,4 @@ private static Object getLogicalTypeValue(Object icebergValue, Schema.FieldType // LocalDateTime, LocalDate, LocalTime return icebergValue; } - - 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(); - } - } } 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 index f8a7d6f2c7d4..9d4274408d97 100644 --- 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.Create; @@ -34,6 +36,7 @@ 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.iceberg.Snapshot; import org.apache.iceberg.Table; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -60,11 +63,11 @@ public PCollection expand(PBegin input) { Table table = TableCache.getRefreshed( scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); - long to = - MoreObjects.firstNonNull(scanConfig.getToSnapshot(), table.currentSnapshot().snapshotId()); PCollection rows = - pollInterval == null ? readBounded(input, to) : readUnbounded(input, pollInterval); + pollInterval == null + ? readBounded(input, table.currentSnapshot()) + : readUnbounded(input, pollInterval); return rows.setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); } @@ -102,7 +105,12 @@ private PCollection readUnbounded(PBegin input, Duration duration) { * Scans a single snapshot range and creates read tasks. Tasks are redistributed and processed * individually using a regular DoFn. */ - private PCollection readBounded(PBegin input, long toSnapshot) { + private PCollection readBounded(PBegin input, @Nullable Snapshot toSnapshot) { + checkStateNotNull( + toSnapshot, + "Table %s does not have any snapshots to read from.", + scanConfig.getTableIdentifier()); + long to = MoreObjects.firstNonNull(scanConfig.getToSnapshot(), toSnapshot.snapshotId()); return input .apply( "Create Single Snapshot Range", @@ -110,7 +118,7 @@ private PCollection readBounded(PBegin input, long toSnapshot) { SnapshotRange.builder() .setTableIdentifierString(scanConfig.getTableIdentifier()) .setFromSnapshotExclusive(scanConfig.getFromSnapshotExclusive()) - .setToSnapshot(toSnapshot) + .setToSnapshot(to) .build())) .apply( "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) 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 index ed5e674ff54f..76af44ceff2a 100644 --- 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 @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.io.iceberg; -import static org.apache.beam.sdk.io.iceberg.ReadFromTasks.getReader; - import java.io.IOException; import java.util.Iterator; import java.util.List; @@ -81,7 +79,7 @@ public void process( return; } FileScanTask task = readTasks.get((int) taskIndex).getFileScanTask(); - try (CloseableIterable reader = getReader(task, table, mapping)) { + try (CloseableIterable reader = ReadUtils.createReader(task, table, mapping)) { for (Record record : reader) { Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); out.output(row); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index be9c0750ac57..f2d751d0616c 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -18,32 +18,18 @@ package org.apache.beam.sdk.io.iceberg; import java.io.IOException; -import java.util.Collection; -import java.util.Map; import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; -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.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.CloseableIterable; -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.parquet.HadoopReadOptions; -import org.apache.parquet.ParquetReadOptions; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -52,8 +38,6 @@ *

For each {@link ReadTask}, reads Iceberg {@link Record}s, and converts to Beam {@link Row}s. */ class ReadFromTasks extends DoFn, Row> { - // default is 8MB. keep this low to avoid overwhelming memory - private static final int MAX_FILE_BUFFER_SIZE = 1 << 20; // 1MB private final IcebergCatalogConfig catalogConfig; ReadFromTasks(IcebergCatalogConfig catalogConfig) { @@ -73,57 +57,11 @@ public void process(@Element KV element, OutputRec FileScanTask task = readTask.getFileScanTask(); - try (CloseableIterable reader = getReader(task, table, mapping)) { + try (CloseableIterable reader = ReadUtils.createReader(task, table, mapping)) { for (Record record : reader) { Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); out.output(row); } } } - - 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 ParquetReader getReader(FileScanTask task, Table table, NameMapping mapping) { - 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 = - IcebergUtils.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); - - return new ParquetReader<>( - inputFile, - table.schema(), - optionsBuilder.build(), - fileSchema -> GenericParquetReaders.buildReader(table.schema(), fileSchema, idToConstants), - mapping, - task.residual(), - false, - true); - } } 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..c94323569a8f --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import java.util.function.BiFunction; +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.Table; +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.parquet.ParquetReader; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PartitionUtil; +import org.apache.parquet.HadoopReadOptions; +import org.apache.parquet.ParquetReadOptions; + +class ReadUtils { + // default is 8MB. keep this low to avoid overwhelming memory + private 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 ParquetReader createReader(FileScanTask task, Table table, NameMapping mapping) { + 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); + + 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(); + } + } +} 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 3de623cc84b5..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 @@ -119,8 +119,7 @@ public boolean advance() throws IOException { DataFile file = fileTask.file(); InputFile input = decryptor.getInputFile(fileTask); Map idToConstants = - IcebergUtils.constantsMap( - fileTask, IdentityPartitionConverters::convertConstant, project); + ReadUtils.constantsMap(fileTask, IdentityPartitionConverters::convertConstant, project); CloseableIterable iterable; switch (file.format()) { 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 index 75f5adbf8656..c9cfe04911b6 100644 --- 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 @@ -33,6 +33,8 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; 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; @@ -68,11 +70,11 @@ public PCollection expand(PBegin input) { private static class SnapshotPollFn extends Watch.Growth.PollFn { private final Gauge latestSnapshot = Metrics.gauge(SnapshotPollFn.class, "latestSnapshot"); private final IcebergScanConfig scanConfig; - private @Nullable Long fromSnapshot; + private @Nullable Long fromSnapshotId; SnapshotPollFn(IcebergScanConfig scanConfig) { this.scanConfig = scanConfig; - this.fromSnapshot = scanConfig.getFromSnapshotExclusive(); + this.fromSnapshotId = scanConfig.getFromSnapshotExclusive(); } @Override @@ -82,25 +84,26 @@ public PollResult apply(String tableIdentifier, Context c) { TableCache.getRefreshed(tableIdentifier, scanConfig.getCatalogConfig().catalog()); Instant timestamp = Instant.now(); - Long currentSnapshot = table.currentSnapshot().snapshotId(); - if (currentSnapshot.equals(fromSnapshot)) { + Snapshot currentSnapshot = table.currentSnapshot(); + if (currentSnapshot == null || Objects.equal(currentSnapshot.snapshotId(), fromSnapshotId)) { // no new snapshot since last poll. return empty result. return getPollResult(null, timestamp); } + Long currentSnapshotId = currentSnapshot.snapshotId(); // if no upper bound is specified, we read up to the current snapshot - Long toSnapshot = MoreObjects.firstNonNull(scanConfig.getSnapshot(), currentSnapshot); + Long toSnapshot = MoreObjects.firstNonNull(scanConfig.getSnapshot(), currentSnapshotId); latestSnapshot.set(toSnapshot); SnapshotRange range = SnapshotRange.builder() - .setFromSnapshotExclusive(fromSnapshot) + .setFromSnapshotExclusive(fromSnapshotId) .setToSnapshot(toSnapshot) .setTableIdentifierString(tableIdentifier) .build(); // update lower bound to current snapshot - fromSnapshot = currentSnapshot; + fromSnapshotId = currentSnapshotId; return getPollResult(range, timestamp); } 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..3ab1f8771753 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 @@ -20,9 +20,13 @@ import static org.apache.beam.sdk.io.iceberg.TestFixtures.createRecord; 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.Iterator; import java.util.List; import java.util.Map; import java.util.UUID; @@ -50,6 +54,7 @@ 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 +68,19 @@ import org.apache.iceberg.types.Types; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; +import org.joda.time.Duration; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; -import org.junit.runners.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); @@ -83,6 +91,14 @@ public class IcebergIOReadTest { @Rule public TestPipeline testPipeline = TestPipeline.create(); + @Parameters + public static Iterable data() { + return Arrays.asList(new Object[][] {{false}, {true}}); + } + + @Parameter(0) + public boolean useIncrementalScan; + static class PrintRow extends DoFn { @ProcessElement @@ -99,27 +115,7 @@ 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()); + commitData(simpleTable); Map catalogProps = ImmutableMap.builder() @@ -133,9 +129,38 @@ public void testSimpleScan() throws Exception { .setCatalogProperties(catalogProps) .build(); + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig).from(tableId); + + List> expectedRecords = + Arrays.asList( + TestFixtures.FILE1SNAPSHOT1, + TestFixtures.FILE2SNAPSHOT1, + TestFixtures.FILE3SNAPSHOT1, + TestFixtures.FILE1SNAPSHOT2, + TestFixtures.FILE2SNAPSHOT2, + TestFixtures.FILE3SNAPSHOT2, + TestFixtures.FILE1SNAPSHOT3, + TestFixtures.FILE2SNAPSHOT3, + TestFixtures.FILE3SNAPSHOT3); + if (useIncrementalScan) { + // only read files that were added in the second snapshot, + // ignoring the first and third snapshots. + expectedRecords = expectedRecords.subList(3, 6); + + Iterator snapshots = simpleTable.snapshots().iterator(); + long first = snapshots.next().snapshotId(); + long second = snapshots.next().snapshotId(); + read = read.fromSnapshotExclusive(first).toSnapshot(second); + } + final List expectedRows = + expectedRecords.stream() + .flatMap(List::stream) + .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) + .collect(Collectors.toList()); + PCollection output = testPipeline - .apply(IcebergIO.readRows(catalogConfig).from(tableId)) + .apply(read) .apply(ParDo.of(new PrintRow())) .setCoder(RowCoder.of(IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); @@ -201,9 +226,13 @@ public void testIdentityColumnScan() throws Exception { .setCatalogProperties(catalogProps) .build(); + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig).from(tableId); + if (useIncrementalScan) { + read = read.toSnapshot(simpleTable.currentSnapshot().snapshotId()); + } PCollection output = testPipeline - .apply(IcebergIO.readRows(catalogConfig).from(tableId)) + .apply(read) .apply(ParDo.of(new PrintRow())) .setCoder(RowCoder.of(IcebergUtils.icebergSchemaToBeamSchema(simpleTable.schema()))); @@ -322,11 +351,12 @@ public void testNameMappingScan() throws Exception { .setCatalogProperties(catalogProps) .build(); + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig).from(tableId); + if (useIncrementalScan) { + read = read.toSnapshot(simpleTable.currentSnapshot().snapshotId()); + } PCollection output = - testPipeline - .apply(IcebergIO.readRows(catalogConfig).from(tableId)) - .apply(ParDo.of(new PrintRow())) - .setCoder(RowCoder.of(beamSchema)); + testPipeline.apply(read).apply(ParDo.of(new PrintRow())).setCoder(RowCoder.of(beamSchema)); final Row[] expectedRows = recordData.stream() @@ -344,6 +374,66 @@ public void testNameMappingScan() throws Exception { testPipeline.run(); } + @Test + public void testUnboundedRead() throws IOException { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = + TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + + commitData(simpleTable); + + 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(); + + List> expectedRecords = + Arrays.asList( + TestFixtures.FILE1SNAPSHOT1, + TestFixtures.FILE2SNAPSHOT1, + TestFixtures.FILE3SNAPSHOT1, + TestFixtures.FILE1SNAPSHOT2, + TestFixtures.FILE2SNAPSHOT2, + TestFixtures.FILE3SNAPSHOT2, + TestFixtures.FILE1SNAPSHOT3, + TestFixtures.FILE2SNAPSHOT3, + TestFixtures.FILE3SNAPSHOT3); + final List expectedRows = + expectedRecords.stream() + .flatMap(List::stream) + .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) + .collect(Collectors.toList()); + + PCollection output = + testPipeline + .apply( + IcebergIO.readRows(catalogConfig) + .from(tableId) + .withTriggeringFrequency(Duration.standardSeconds(1)) + .toSnapshot(simpleTable.currentSnapshot().snapshotId())) + .apply(ParDo.of(new PrintRow())) + .setCoder(RowCoder.of(IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); + + assertEquals(PCollection.IsBounded.UNBOUNDED, output.isBounded()); + PAssert.that(output) + .satisfies( + (Iterable rows) -> { + assertThat(rows, containsInAnyOrder(expectedRows.toArray())); + return null; + }); + + testPipeline.run(); + } + @SuppressWarnings("unchecked") public static GenericRecord avroGenericRecord( org.apache.avro.Schema schema, Map values) { @@ -396,4 +486,48 @@ private static File createTempFile() throws IOException { boolean unused = tempFile.delete(); return tempFile; } + + private void commitData(Table simpleTable) throws IOException { + // first snapshot + 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(); + + // second snapshot + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "file1s2.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT2)) + .appendFile( + warehouse.writeRecords( + "file2s2.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT2)) + .appendFile( + warehouse.writeRecords( + "file3s2.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT2)) + .commit(); + + // third snapshot + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "file1s3.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT3)) + .appendFile( + warehouse.writeRecords( + "file2s3.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT3)) + .appendFile( + warehouse.writeRecords( + "file3s3.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT3)) + .commit(); + } } 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 c62899a341a0..f319abe5f902 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 @@ -22,12 +22,14 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; +import java.io.IOException; +import java.util.Arrays; import java.util.HashMap; +import java.util.Iterator; import java.util.List; 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; @@ -36,14 +38,21 @@ import org.apache.beam.sdk.values.PCollectionRowTuple; import org.apache.beam.sdk.values.Row; import org.apache.iceberg.CatalogUtil; +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.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameter; +import org.junit.runners.Parameterized.Parameters; import org.yaml.snakeyaml.Yaml; +@RunWith(Parameterized.class) public class IcebergReadSchemaTransformProviderTest { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -52,20 +61,35 @@ public class IcebergReadSchemaTransformProviderTest { @Rule public TestPipeline testPipeline = TestPipeline.create(); + @Parameters + public static Iterable data() { + return Arrays.asList(new Object[][] {{false}, {true}}); + } + + @Parameter(0) + public boolean useIncrementalScan; + @Test public void testBuildTransformWithRow() { Map properties = new HashMap<>(); properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", "test_location"); - Row transformConfigRow = + Row.FieldValueBuilder configBuilder = Row.withSchema(new IcebergReadSchemaTransformProvider().configurationSchema()) .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_name", "test-name") - .withFieldValue("catalog_properties", properties) - .build(); + .withFieldValue("catalog_properties", properties); - new IcebergReadSchemaTransformProvider().from(transformConfigRow); + if (useIncrementalScan) { + configBuilder = + configBuilder + .withFieldValue("from_snapshot_exclusive", 123L) + .withFieldValue("to_snapshot", 456L) + .withFieldValue("triggering_frequency_seconds", 789); + } + + new IcebergReadSchemaTransformProvider().from(configBuilder.build()); } @Test @@ -76,42 +100,48 @@ 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()); + commitData(simpleTable); Map properties = new HashMap<>(); properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", warehouse.location); - Configuration readConfig = + Configuration.Builder readConfigBuilder = Configuration.builder() .setTable(identifier) .setCatalogName("name") - .setCatalogProperties(properties) - .build(); + .setCatalogProperties(properties); + + List> expectedRecords = + Arrays.asList( + TestFixtures.FILE1SNAPSHOT1, + TestFixtures.FILE2SNAPSHOT1, + TestFixtures.FILE3SNAPSHOT1, + TestFixtures.FILE1SNAPSHOT2, + TestFixtures.FILE2SNAPSHOT2, + TestFixtures.FILE3SNAPSHOT2, + TestFixtures.FILE1SNAPSHOT3, + TestFixtures.FILE2SNAPSHOT3, + TestFixtures.FILE3SNAPSHOT3); + if (useIncrementalScan) { + // only read files that were added in the second snapshot, + // ignoring the first and third snapshots. + expectedRecords = expectedRecords.subList(3, 6); + + Iterator snapshots = simpleTable.snapshots().iterator(); + long first = snapshots.next().snapshotId(); + long second = snapshots.next().snapshotId(); + readConfigBuilder = readConfigBuilder.setFromSnapshotExclusive(first).setToSnapshot(second); + } + 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) @@ -132,27 +162,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()); + commitData(simpleTable); String yamlConfig = String.format( @@ -162,8 +172,36 @@ public void testReadUsingManagedTransform() throws Exception { + " type: %s\n" + " warehouse: %s", identifier, CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, warehouse.location); - Map configMap = new Yaml().load(yamlConfig); + List> expectedRecords = + Arrays.asList( + TestFixtures.FILE1SNAPSHOT1, + TestFixtures.FILE2SNAPSHOT1, + TestFixtures.FILE3SNAPSHOT1, + TestFixtures.FILE1SNAPSHOT2, + TestFixtures.FILE2SNAPSHOT2, + TestFixtures.FILE3SNAPSHOT2, + TestFixtures.FILE1SNAPSHOT3, + TestFixtures.FILE2SNAPSHOT3, + TestFixtures.FILE3SNAPSHOT3); + if (useIncrementalScan) { + // only read files that were added in the second snapshot, + // ignoring the first and third snapshots. + expectedRecords = expectedRecords.subList(3, 6); + + Iterator snapshots = simpleTable.snapshots().iterator(); + long first = snapshots.next().snapshotId(); + long second = snapshots.next().snapshotId(); + yamlConfig += + String.format("\n" + "from_snapshot_exclusive: %s\n" + "to_snapshot: %s", first, second); + } + 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)) @@ -178,4 +216,48 @@ public void testReadUsingManagedTransform() throws Exception { testPipeline.run(); } + + private void commitData(Table simpleTable) throws IOException { + // first snapshot + 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(); + + // second snapshot + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "file1s2.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT2)) + .appendFile( + warehouse.writeRecords( + "file2s2.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT2)) + .appendFile( + warehouse.writeRecords( + "file3s2.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT2)) + .commit(); + + // third snapshot + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "file1s3.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT3)) + .appendFile( + warehouse.writeRecords( + "file2s3.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT3)) + .appendFile( + warehouse.writeRecords( + "file3s3.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT3)) + .commit(); + } } 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..768d6865b759 --- /dev/null +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/ReadUtilsTest.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.iceberg; + +import static org.junit.Assert.assertEquals; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; +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.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +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.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +/** 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"); + + @Test + public void testCreateReader() throws IOException { + TableIdentifier tableId = + TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + 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())); + ParquetReader reader = ReadUtils.createReader(fileScanTask, simpleTable, null); + List recordsRead = new ArrayList<>(); + reader.forEach(recordsRead::add); + + assertEquals(data.get(fileName), recordsRead); + numFiles++; + } + } + } + assertEquals(data.size(), numFiles); + } +} 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..b55f98065bb2 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 @@ -282,6 +282,11 @@ public Record apply(Row input) { /** Populates the Iceberg table and Returns a {@link List} of expected elements. */ private 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)); @@ -307,6 +312,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); @@ -385,6 +394,44 @@ public void testRead() throws Exception { pipeline.run().waitUntilFinish(); } + @Test + public void testUnboundedRead() throws Exception { + Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + + List expectedRows = populateTable(table); + + Map config = new HashMap<>(managedIcebergConfig(tableId())); + config.put("triggering_frequency_seconds", 30); + config.put("to_snapshot", table.currentSnapshot().snapshotId()); + + PCollection rows = + pipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); + + PAssert.that(rows).containsInAnyOrder(expectedRows); + pipeline.run().waitUntilFinish(); + } + + @Test + public void testReadSnapshotRange() throws Exception { + Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + + populateTable(table, "a"); + long from = table.currentSnapshot().snapshotId(); + List expectedRows = populateTable(table, "b"); + long to = table.currentSnapshot().snapshotId(); + populateTable(table, "c"); + + Map config = new HashMap<>(managedIcebergConfig(tableId())); + config.put("from_snapshot_exclusive", from); + config.put("to_snapshot", to); + + PCollection rows = + pipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); + + PAssert.that(rows).containsInAnyOrder(expectedRows); + pipeline.run().waitUntilFinish(); + } + @Test public void testWrite() throws IOException { // Write with Beam From 622625f5666f5267429f6a4ad909c85e260db42d Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Sun, 9 Feb 2025 22:42:53 -0500 Subject: [PATCH 09/32] add a java doc; don't use static block to create coder --- sdks/java/io/iceberg/bqms/build.gradle | 4 +--- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 2 +- .../sdk/io/iceberg/IncrementalScanSource.java | 7 ++++--- .../apache/beam/sdk/io/iceberg/ReadTask.java | 17 ++++++++++------- .../sdk/io/iceberg/ReadTaskDescriptor.java | 18 +++++++++++------- .../beam/sdk/io/iceberg/SnapshotRange.java | 3 +++ 6 files changed, 30 insertions(+), 21 deletions(-) diff --git a/sdks/java/io/iceberg/bqms/build.gradle b/sdks/java/io/iceberg/bqms/build.gradle index 3d268b9a8271..e42aafc5f424 100644 --- a/sdks/java/io/iceberg/bqms/build.gradle +++ b/sdks/java/io/iceberg/bqms/build.gradle @@ -21,9 +21,7 @@ plugins { applyJavaNature( automaticModuleName: 'org.apache.beam.sdk.io.iceberg.bqms', - shadowClosure: { - relocate "com.google.auth", getJavaRelocatedPath("bqms.com.google.auth") - }, + shadowClosure: {}, exportJavadoc: false, publish: false, // it's an intermediate jar for io-expansion-service validateShadowJar: false 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 ca8bd64ca07a..9d626536edec 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 @@ -116,7 +116,7 @@ * * {@code to_snapshot} * {@code long} - * For the source; Reads up to this snapshot ID (inclusive). If unset and the source is bounded, it will read + * For the source; reads up to this snapshot ID (inclusive). If unset and the source is bounded, it will read * up to the current snapshot (inclusive). If unset and source is unbounded, it will continue polling for new snapshots forever. * * 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 index 9d4274408d97..e72249831fb7 100644 --- 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 @@ -80,7 +80,7 @@ private PCollection readUnbounded(PBegin input, Duration duration) { .apply("Watch for Snapshots", new WatchForSnapshots(scanConfig, duration)) .apply( "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) - .setCoder(KvCoder.of(ReadTaskDescriptor.CODER, ReadTask.CODER)) + .setCoder(KvCoder.of(ReadTaskDescriptor.getCoder(), ReadTask.getCoder())) .apply( "Apply User Trigger", Window.>into(new GlobalWindows()) @@ -95,7 +95,8 @@ private PCollection readUnbounded(PBegin input, Duration duration) { .withShardedKey()) .setCoder( KvCoder.of( - ShardedKey.Coder.of(ReadTaskDescriptor.CODER), IterableCoder.of(ReadTask.CODER))) + ShardedKey.Coder.of(ReadTaskDescriptor.getCoder()), + IterableCoder.of(ReadTask.getCoder()))) .apply( "Read Rows From Grouped Tasks", ParDo.of(new ReadFromGroupedTasks(scanConfig.getCatalogConfig()))); @@ -122,7 +123,7 @@ private PCollection readBounded(PBegin input, @Nullable Snapshot toSnapshot .build())) .apply( "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) - .setCoder(KvCoder.of(ReadTaskDescriptor.CODER, ReadTask.CODER)) + .setCoder(KvCoder.of(ReadTaskDescriptor.getCoder(), ReadTask.getCoder())) .apply(Redistribute.arbitrarily()) .apply("Read Rows From Tasks", ParDo.of(new ReadFromTasks(scanConfig.getCatalogConfig()))); } 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 index 964dde36608f..da09c8ce1395 100644 --- 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 @@ -18,9 +18,9 @@ package org.apache.beam.sdk.io.iceberg; import com.google.auto.value.AutoValue; -import org.apache.beam.sdk.coders.Coder; 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; @@ -31,14 +31,17 @@ @DefaultSchema(AutoValueSchema.class) @AutoValue abstract class ReadTask { - static final Coder CODER; + private static @MonotonicNonNull SchemaCoder CODER; - static { - try { - CODER = SchemaRegistry.createDefault().getSchemaCoder(ReadTask.class); - } catch (NoSuchSchemaException e) { - throw new RuntimeException(e); + 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; 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 index 4474a893c215..86ae44c9eca9 100644 --- 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 @@ -18,24 +18,28 @@ package org.apache.beam.sdk.io.iceberg; import com.google.auto.value.AutoValue; -import org.apache.beam.sdk.coders.Coder; 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 and snapshot a {@link ReadTask} belongs to. */ @DefaultSchema(AutoValueSchema.class) @AutoValue abstract class ReadTaskDescriptor { - static final Coder CODER; + private static @MonotonicNonNull SchemaCoder CODER; - static { - try { - CODER = SchemaRegistry.createDefault().getSchemaCoder(ReadTaskDescriptor.class); - } catch (NoSuchSchemaException e) { - throw new RuntimeException(e); + 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() { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java index 9ba0c04d22ad..4c49f53043e8 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java @@ -37,6 +37,9 @@ static Builder builder() { abstract String getTableIdentifierString(); + /** + * Snapshot to start reading from (exclusive). If null, starts reading from the oldest snapshot. + */ abstract @Nullable Long getFromSnapshotExclusive(); abstract long getToSnapshot(); From 4c25d3f4319ffd1a203e26c8575b533991274e4a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Sun, 9 Feb 2025 23:39:06 -0500 Subject: [PATCH 10/32] spotless --- .../java/org/apache/beam/sdk/io/iceberg/ReadTask.java | 8 ++++---- .../apache/beam/sdk/io/iceberg/ReadTaskDescriptor.java | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) 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 index da09c8ce1395..220b46b9fdb7 100644 --- 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 @@ -31,17 +31,17 @@ @DefaultSchema(AutoValueSchema.class) @AutoValue abstract class ReadTask { - private static @MonotonicNonNull SchemaCoder CODER; + private static @MonotonicNonNull SchemaCoder coder; static SchemaCoder getCoder() { - if (CODER == null) { + if (coder == null) { try { - CODER = SchemaRegistry.createDefault().getSchemaCoder(ReadTask.class); + coder = SchemaRegistry.createDefault().getSchemaCoder(ReadTask.class); } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } } - return CODER; + return coder; } private transient @MonotonicNonNull FileScanTask cachedFileScanTask; 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 index 86ae44c9eca9..2e5c37fdcc19 100644 --- 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 @@ -29,17 +29,17 @@ @DefaultSchema(AutoValueSchema.class) @AutoValue abstract class ReadTaskDescriptor { - private static @MonotonicNonNull SchemaCoder CODER; + private static @MonotonicNonNull SchemaCoder coder; static SchemaCoder getCoder() { - if (CODER == null) { + if (coder == null) { try { - CODER = SchemaRegistry.createDefault().getSchemaCoder(ReadTaskDescriptor.class); + coder = SchemaRegistry.createDefault().getSchemaCoder(ReadTaskDescriptor.class); } catch (NoSuchSchemaException e) { throw new RuntimeException(e); } } - return CODER; + return coder; } static Builder builder() { From 86661664ba458dc9420fa337b92f3f876265067e Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 13 Feb 2025 12:35:52 -0500 Subject: [PATCH 11/32] add options: from/to timestamp, starting strategy, and streaming toggle --- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 300 +++++++++++---- .../IcebergReadSchemaTransformProvider.java | 78 +++- .../sdk/io/iceberg/IcebergScanConfig.java | 61 ++++ .../sdk/io/iceberg/IncrementalScanSource.java | 37 +- .../apache/beam/sdk/io/iceberg/ReadUtils.java | 44 +++ .../sdk/io/iceberg/WatchForSnapshots.java | 16 +- .../sdk/io/iceberg/IcebergIOReadTest.java | 342 ++++++++++-------- ...cebergReadSchemaTransformProviderTest.java | 107 +----- .../sdk/io/iceberg/TestDataWarehouse.java | 52 +++ .../beam/sdk/io/iceberg/TestFixtures.java | 58 ++- .../catalog/BigQueryMetastoreCatalogIT.java | 4 +- .../iceberg/catalog/IcebergCatalogBaseIT.java | 137 +++++-- 12 files changed, 838 insertions(+), 398 deletions(-) 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 9d626536edec..46a1300acbe1 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,25 @@
  *         "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(...));
  * }
* - *

Configuration Options

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

Configuration Options

+ * + *
* * * @@ -81,17 +84,6 @@ * template to use dynamic destinations (see the `Dynamic Destinations` section below for details). * * - * - * - * - * - * * * * @@ -106,33 +98,92 @@ * implementation, but this list * is a good starting point. * + *
Parameter Type Description
{@code triggering_frequency_seconds} {@code int} - *

Sink: 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. - *

Source: Enables streaming reads. Roughly every {@code triggering_frequency_seconds} duration, the source - * will scan the table for new snapshots and read new records. - *

{@code catalog_name} {@code str} The name of the catalog. Defaults to {@code apache-beam-}.
+ * + *

Sink-only Options

+ * + * + * + * + * * - * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
Parameter Type Description
{@code from_snapshot_exclusive} {@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).
+ * + *

Source-only Options

+ * + * + * + * + * + * + * * - * * * * * - * * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * *
Parameter Type Description
{@code from_snapshot} {@code long} For the source; starts reading from this snapshot ID (exclusive). If unset, it will start reading from the - * oldest snapshot (inclusive). + * Starts reading from this snapshot ID (inclusive). *
{@code to_snapshot} {@code long} For the source; reads up to this snapshot ID (inclusive). If unset and the source is bounded, it will read + * Reads up to this snapshot ID (inclusive). If unset and the source is bounded, it will read * up to the current snapshot (inclusive). If unset and source is unbounded, it 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). + *
{@code streaming} {@code boolean} Enables streaming reads. By default, the streaming source will start reading from the latest + * snapshot (inclusive) and continue polling forever based on the specified {@code poll_interval_seconds}. + *
{@code poll_interval_seconds} {@code int} + * The interval at which to scan the table for new snapshots. Defaults to 60 seconds. For streaming reads only. + *
{@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. + *

* - *

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

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. - * *

Beam Rows

* *

Being a Managed transform, this IO exclusively writes and reads using Beam {@link Row}s. @@ -216,6 +267,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 @@ -232,23 +293,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): * @@ -287,10 +333,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
@@ -329,7 +376,91 @@
  *   
  * 
  *
- * 

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

Reading from Tables

+ * + * A simple batch read from an Iceberg table looks like this: + * + *
{@code
+ * Map config = Map.of(
+ *         "table", table,
+ *         "catalog_name", name,
+ *         "catalog_properties", Map.of(...),
+ *         "config_properties", Map.of(...));
+ *
+ * PCollection = pipeline
+ *     .apply(Managed.read(ICEBERG).withConfig(config))
+ *     .getSinglePCollection();
+ * }
+ * + * Setting {@code streaming} to {@code true} enables streaming read mode, which continuously + * polls the table for new snapshots. The default polling interval is 60 seconds, but this can be + * overridden using {@code poll_interval_seconds}: + * + *
{@code
+ * config.put("streaming", true);
+ * config.put("poll_interval_seconds", 10);
+ * }
+ * + *

Choosing a Starting Point

+ * + * 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).withConfig(config))
+ *     .getSinglePCollection();
+ * }
+ * + *

Choosing an End Point

+ * + * 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 also be overridden with a + * couple of mutually exclusive 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).withConfig(config))
+ *     .getSinglePCollection();
+ * }
+ * + * Note: An end point can also be set when performing a streaming read. */ @Internal public class IcebergIO { @@ -419,16 +550,28 @@ public static ReadRows readRows(IcebergCatalogConfig catalogConfig) { @AutoValue public abstract static class ReadRows extends PTransform> { + public enum StartingStrategy { + EARLIEST, + LATEST + } abstract IcebergCatalogConfig getCatalogConfig(); abstract @Nullable TableIdentifier getTableIdentifier(); - abstract @Nullable Long getFromSnapshotExclusive(); + abstract @Nullable Long getFromSnapshot(); abstract @Nullable Long getToSnapshot(); - abstract @Nullable Duration getTriggeringFrequency(); + abstract @Nullable Long getFromTimestamp(); + + abstract @Nullable Long getToTimestamp(); + + abstract @Nullable Duration getPollInterval(); + + abstract @Nullable Boolean getStreaming(); + + abstract @Nullable StartingStrategy getStartingStrategy(); abstract Builder toBuilder(); @@ -438,11 +581,19 @@ abstract static class Builder { abstract Builder setTableIdentifier(TableIdentifier identifier); - abstract Builder setFromSnapshotExclusive(@Nullable Long fromSnapshotExclusive); + abstract Builder setFromSnapshot(@Nullable Long fromSnapshot); abstract Builder setToSnapshot(@Nullable Long toSnapshot); - abstract Builder setTriggeringFrequency(Duration triggeringFrequency); + abstract Builder setFromTimestamp(@Nullable Long fromTimestamp); + + abstract Builder setToTimestamp(@Nullable Long toTimestamp); + + abstract Builder setPollInterval(@Nullable Duration triggeringFrequency); + + abstract Builder setStreaming(@Nullable Boolean streaming); + + abstract Builder setStartingStrategy(@Nullable StartingStrategy strategy); abstract ReadRows build(); } @@ -451,16 +602,32 @@ public ReadRows from(TableIdentifier tableIdentifier) { return toBuilder().setTableIdentifier(tableIdentifier).build(); } - public ReadRows fromSnapshotExclusive(@Nullable Long fromSnapshotExclusive) { - return toBuilder().setFromSnapshotExclusive(fromSnapshotExclusive).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 withTriggeringFrequency(Duration triggeringFrequency) { - return toBuilder().setTriggeringFrequency(triggeringFrequency).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 @@ -476,14 +643,19 @@ public PCollection expand(PBegin input) { .setScanType(IcebergScanConfig.ScanType.TABLE) .setTableIdentifier(tableId) .setSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())) - .setFromSnapshotExclusive(getFromSnapshotExclusive()) + .setFromSnapshotInclusive(getFromSnapshot()) .setToSnapshot(getToSnapshot()) + .setFromTimestamp(getFromTimestamp()) + .setToTimestamp(getToTimestamp()) + .setStartingStrategy(getStartingStrategy()) + .setStreaming(getStreaming()) + .setPollInterval(getPollInterval()) .build(); - if (getTriggeringFrequency() != null - || scanConfig.getToSnapshot() != null - || scanConfig.getFromSnapshotExclusive() != null) { + scanConfig.validate(); + + if (scanConfig.useIncrementalSource()) { return input - .apply(new IncrementalScanSource(scanConfig, getTriggeringFrequency())) + .apply(new IncrementalScanSource(scanConfig)) .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); } 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 e5350ce58ce2..aa92e7af86e4 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 @@ -22,10 +22,12 @@ 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; @@ -37,6 +39,8 @@ 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; @@ -89,16 +93,32 @@ Row getConfigurationRow() { @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()) .from(TableIdentifier.parse(configuration.getTable())) - .fromSnapshotExclusive(configuration.getFromSnapshotExclusive()) - .toSnapshot(configuration.getToSnapshot()); - - @Nullable Integer triggeringFrequencySeconds = configuration.getTriggeringFrequencySeconds(); - if (triggeringFrequencySeconds != null) { - readRows = - readRows.withTriggeringFrequency(Duration.standardSeconds(triggeringFrequencySeconds)); + .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); @@ -129,19 +149,33 @@ static Builder builder() { @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( - "The frequency at which to poll for new snapshots. An unbounded source is used when this is set.") - abstract @Nullable Integer getTriggeringFrequencySeconds(); + "Starts reading from the first snapshot (inclusive) that was created after this timestamp (in milliseconds).") + abstract @Nullable Long getFromTimestamp(); @SchemaFieldDescription( - "Starts reading from this snapshot ID (exclusive). If unset, the source will " - + "start reading from the oldest snapshot (inclusive).") - abstract @Nullable Long getFromSnapshotExclusive(); + "Reads up to the latest snapshot (inclusive) created before this timestamp (in milliseconds).") + abstract @Nullable Long getToTimestamp(); @SchemaFieldDescription( - "Reads up to this snapshot ID (inclusive). If unset and the source is bounded, it will read up to the current snapshot. " - + "The unbounded source will continue polling for new snapshots forever.") - abstract @Nullable Long getToSnapshot(); + "The interval at which to poll for new snapshots. Defaults to 60 seconds.") + abstract @Nullable Integer getPollIntervalSeconds(); + + @SchemaFieldDescription( + "Enables streaming reads. By default, the streaming source will start reading from the " + + "latest snapshot (inclusive) and continue polling forever based on the specified poll_interval_seconds") + abstract @Nullable Boolean getStreaming(); + + @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(); @AutoValue.Builder abstract static class Builder { @@ -153,12 +187,20 @@ abstract static class Builder { abstract Builder setConfigProperties(Map confProperties); - abstract Builder setTriggeringFrequencySeconds(Integer snapshot); - - abstract Builder setFromSnapshotExclusive(Long snapshot); + abstract Builder setFromSnapshot(Long snapshot); abstract Builder setToSnapshot(Long snapshot); + abstract Builder setFromTimestamp(Long timestamp); + + abstract Builder setToTimestamp(Long timestamp); + + abstract Builder setPollIntervalSeconds(Integer pollInterval); + + abstract Builder setStreaming(Boolean streaming); + + abstract Builder setStartingStrategy(String strategy); + abstract Configuration 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..7e1d3f98e825 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 @@ -19,7 +19,10 @@ import com.google.auto.value.AutoValue; import java.io.Serializable; +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.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.ImmutableMap; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -27,9 +30,17 @@ 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 { + public boolean useIncrementalSource() { + return MoreObjects.firstNonNull(getStreaming(), false) + || getToTimestamp() != null + || getFromSnapshotInclusive() != null + || getToSnapshot() != null + || getStartingStrategy() != null; + } private transient @MonotonicNonNull Table cachedTable; @@ -92,6 +103,21 @@ 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 Boolean getStreaming(); + + @Pure + public abstract @Nullable Duration getPollInterval(); + + @Pure + public abstract @Nullable StartingStrategy getStartingStrategy(); + @Pure public abstract @Nullable String getTag(); @@ -113,6 +139,11 @@ public static Builder builder() { .setFromSnapshotRefExclusive(null) .setToSnapshot(null) .setToSnapshotRef(null) + .setFromTimestamp(null) + .setToTimestamp(null) + .setStreaming(null) + .setPollInterval(null) + .setStartingStrategy(null) .setTag(null) .setBranch(null); } @@ -157,10 +188,40 @@ 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 setStreaming(@Nullable Boolean streaming); + + public abstract Builder setPollInterval(@Nullable Duration pollInterval); + + public abstract Builder setStartingStrategy(@Nullable StartingStrategy strategy); + public abstract Builder setTag(@Nullable String tag); public abstract Builder setBranch(@Nullable String branch); public abstract IcebergScanConfig build(); } + + void validate() { + if (getStartingStrategy() != null) { + Preconditions.checkArgument( + getFromTimestamp() == null && getFromSnapshotInclusive() == null, + "Invalid source configuration: 'from_timestamp' and 'from_snapshot' are not allowed when 'starting_strategy' is set"); + } + Preconditions.checkArgument( + getFromTimestamp() == null || getFromSnapshotInclusive() == null, + "Invalid source configuration: Only one of 'from_timestamp' or 'from_snapshot' can be set"); + Preconditions.checkArgument( + getToTimestamp() == null || getToSnapshot() == null, + "Invalid source configuration: Only one of 'to_timestamp' or 'to_snapshot' can be set"); + + if (getPollInterval() != null) { + Preconditions.checkArgument( + Boolean.TRUE.equals(getStreaming()), + "Invalid source configuration: 'poll_interval_seconds' can only be set when streaming is true"); + } + } } 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 index e72249831fb7..064abe1a0a9f 100644 --- 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 @@ -36,7 +36,6 @@ 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.iceberg.Snapshot; import org.apache.iceberg.Table; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; @@ -48,14 +47,12 @@ * unbounded implementations) is used to process each task and output Beam rows. */ class IncrementalScanSource extends PTransform> { - // For the unbounded implementation. private static final long MAX_FILES_BATCH_BYTE_SIZE = 1L << 32; // 4 GB - private final @Nullable Duration pollInterval; + private static final Duration DEFAULT_POLL_INTERVAL = Duration.standardSeconds(60); private final IcebergScanConfig scanConfig; - IncrementalScanSource(IcebergScanConfig scanConfig, @Nullable Duration pollInterval) { + IncrementalScanSource(IcebergScanConfig scanConfig) { this.scanConfig = scanConfig; - this.pollInterval = pollInterval; } @Override @@ -65,9 +62,9 @@ public PCollection expand(PBegin input) { scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); PCollection rows = - pollInterval == null - ? readBounded(input, table.currentSnapshot()) - : readUnbounded(input, pollInterval); + MoreObjects.firstNonNull(scanConfig.getStreaming(), false) + ? readUnbounded(input) + : readBounded(input, table); return rows.setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); } @@ -75,9 +72,12 @@ public PCollection expand(PBegin input) { * Watches for new snapshots and creates tasks for each range. Using GiB for autosharding, this * groups tasks in batches of up to 4GB, then reads from each batch using an SDF. */ - private PCollection readUnbounded(PBegin input, Duration duration) { + private PCollection readUnbounded(PBegin input) { + @Nullable + Duration pollInterval = + MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); return input - .apply("Watch for Snapshots", new WatchForSnapshots(scanConfig, duration)) + .apply("Watch for Snapshots", new WatchForSnapshots(scanConfig, pollInterval)) .apply( "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) .setCoder(KvCoder.of(ReadTaskDescriptor.getCoder(), ReadTask.getCoder())) @@ -86,12 +86,12 @@ private PCollection readUnbounded(PBegin input, Duration duration) { Window.>into(new GlobalWindows()) .triggering( Repeatedly.forever( - AfterProcessingTime.pastFirstElementInPane().plusDelayOf(duration))) + AfterProcessingTime.pastFirstElementInPane().plusDelayOf(pollInterval))) .discardingFiredPanes()) .apply( GroupIntoBatches.ofByteSize( MAX_FILES_BATCH_BYTE_SIZE, ReadTask::getByteSize) - .withMaxBufferingDuration(duration) + .withMaxBufferingDuration(pollInterval) .withShardedKey()) .setCoder( KvCoder.of( @@ -106,19 +106,24 @@ private PCollection readUnbounded(PBegin input, Duration duration) { * Scans a single snapshot range and creates read tasks. Tasks are redistributed and processed * individually using a regular DoFn. */ - private PCollection readBounded(PBegin input, @Nullable Snapshot toSnapshot) { + private PCollection readBounded(PBegin input, Table table) { checkStateNotNull( - toSnapshot, + table.currentSnapshot().snapshotId(), "Table %s does not have any snapshots to read from.", scanConfig.getTableIdentifier()); - long to = MoreObjects.firstNonNull(scanConfig.getToSnapshot(), toSnapshot.snapshotId()); + + @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 Single Snapshot Range", Create.of( SnapshotRange.builder() .setTableIdentifierString(scanConfig.getTableIdentifier()) - .setFromSnapshotExclusive(scanConfig.getFromSnapshotExclusive()) + .setFromSnapshotExclusive(from) .setToSnapshot(to) .build())) .apply( 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 index c94323569a8f..162f22262f2f 100644 --- 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 @@ -22,6 +22,8 @@ import java.util.Map; import java.util.Set; import java.util.function.BiFunction; +import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; +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.Sets; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.FileScanTask; @@ -40,8 +42,10 @@ 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; class ReadUtils { // default is 8MB. keep this low to avoid overwhelming memory @@ -106,4 +110,44 @@ static ParquetReader createReader(FileScanTask task, Table table, NameMa 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)) { + fromSnapshot = table.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; + } } 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 index c9cfe04911b6..d6679a156025 100644 --- 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 @@ -74,7 +74,6 @@ private static class SnapshotPollFn extends Watch.Growth.PollFn 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); + } Instant timestamp = Instant.now(); Snapshot currentSnapshot = table.currentSnapshot(); if (currentSnapshot == null || Objects.equal(currentSnapshot.snapshotId(), fromSnapshotId)) { // no new snapshot since last poll. return empty result. - return getPollResult(null, timestamp); + return getPollResult(null, timestamp, isComplete); } Long currentSnapshotId = currentSnapshot.snapshotId(); // if no upper bound is specified, we read up to the current snapshot - Long toSnapshot = MoreObjects.firstNonNull(scanConfig.getSnapshot(), currentSnapshotId); + Long toSnapshot = MoreObjects.firstNonNull(userSpecifiedToSnapshot, currentSnapshotId); latestSnapshot.set(toSnapshot); SnapshotRange range = @@ -105,18 +109,18 @@ public PollResult apply(String tableIdentifier, Context c) { // update lower bound to current snapshot fromSnapshotId = currentSnapshotId; - return getPollResult(range, timestamp); + return getPollResult(range, timestamp, isComplete); } /** Returns an appropriate PollResult based on the requested boundedness. */ private PollResult getPollResult( - @Nullable SnapshotRange range, Instant timestamp) { + @Nullable SnapshotRange range, Instant timestamp, boolean isComplete) { List> timestampedValues = range == null ? Collections.emptyList() : Collections.singletonList(TimestampedValue.of(range, timestamp)); - return scanConfig.getToSnapshot() != null + return isComplete ? PollResult.complete(timestampedValues) // stop at specified snapshot : PollResult.incomplete(timestampedValues); // continue forever } 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 3ab1f8771753..986197d0ec20 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 @@ -26,7 +26,6 @@ import java.io.File; import java.io.IOException; import java.util.Arrays; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.UUID; @@ -34,16 +33,18 @@ 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.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; @@ -68,11 +69,14 @@ 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.Parameterized; import org.junit.runners.Parameterized.Parameter; @@ -90,6 +94,8 @@ 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(); @Parameters public static Iterable data() { @@ -108,6 +114,67 @@ public void process(@Element Row row, OutputReceiver output) throws Excepti } } + @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).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()).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()) + .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()) + .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 = @@ -115,42 +182,12 @@ public void testSimpleScan() throws Exception { Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); - commitData(simpleTable); + List> expectedRecords = warehouse.commitData(simpleTable); - Map catalogProps = - ImmutableMap.builder() - .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .put("warehouse", warehouse.location) - .build(); + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId); - IcebergCatalogConfig catalogConfig = - IcebergCatalogConfig.builder() - .setCatalogName("name") - .setCatalogProperties(catalogProps) - .build(); - - IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig).from(tableId); - - List> expectedRecords = - Arrays.asList( - TestFixtures.FILE1SNAPSHOT1, - TestFixtures.FILE2SNAPSHOT1, - TestFixtures.FILE3SNAPSHOT1, - TestFixtures.FILE1SNAPSHOT2, - TestFixtures.FILE2SNAPSHOT2, - TestFixtures.FILE3SNAPSHOT2, - TestFixtures.FILE1SNAPSHOT3, - TestFixtures.FILE2SNAPSHOT3, - TestFixtures.FILE3SNAPSHOT3); if (useIncrementalScan) { - // only read files that were added in the second snapshot, - // ignoring the first and third snapshots. - expectedRecords = expectedRecords.subList(3, 6); - - Iterator snapshots = simpleTable.snapshots().iterator(); - long first = snapshots.next().snapshotId(); - long second = snapshots.next().snapshotId(); - read = read.fromSnapshotExclusive(first).toSnapshot(second); + read = read.toSnapshot(simpleTable.currentSnapshot().snapshotId()); } final List expectedRows = expectedRecords.stream() @@ -159,10 +196,7 @@ public void testSimpleScan() throws Exception { .collect(Collectors.toList()); PCollection output = - testPipeline - .apply(read) - .apply(ParDo.of(new PrintRow())) - .setCoder(RowCoder.of(IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); + testPipeline.apply(read).apply(ParDo.of(new PrintRow())).setRowSchema(schema); PAssert.that(output) .satisfies( @@ -214,27 +248,12 @@ 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); + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId); if (useIncrementalScan) { read = read.toSnapshot(simpleTable.currentSnapshot().snapshotId()); } PCollection output = - testPipeline - .apply(read) - .apply(ParDo.of(new PrintRow())) - .setCoder(RowCoder.of(IcebergUtils.icebergSchemaToBeamSchema(simpleTable.schema()))); + testPipeline.apply(read).apply(ParDo.of(new PrintRow())).setRowSchema(schema); PAssert.that(output) .satisfies( @@ -339,24 +358,12 @@ 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(); - - IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig).from(tableId); + IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId); if (useIncrementalScan) { read = read.toSnapshot(simpleTable.currentSnapshot().snapshotId()); } PCollection output = - testPipeline.apply(read).apply(ParDo.of(new PrintRow())).setCoder(RowCoder.of(beamSchema)); + testPipeline.apply(read).apply(ParDo.of(new PrintRow())).setRowSchema(beamSchema); final Row[] expectedRows = recordData.stream() @@ -375,62 +382,122 @@ public void testNameMappingScan() throws Exception { } @Test - public void testUnboundedRead() throws IOException { + 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", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); - final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); + Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); - commitData(simpleTable); + 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()); - Map catalogProps = - ImmutableMap.builder() - .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) - .put("warehouse", warehouse.location) - .build(); + IcebergIO.ReadRows readRows = + IcebergIO.readRows(catalogConfig()) + .from(tableId) + .streaming(streaming) + .toSnapshot(simpleTable.currentSnapshot().snapshotId()); + if (strategy != null) { + readRows = readRows.withStartingStrategy(strategy); + } - IcebergCatalogConfig catalogConfig = - IcebergCatalogConfig.builder() - .setCatalogName("name") - .setCatalogProperties(catalogProps) - .build(); + PCollection output = testPipeline.apply(readRows); + + PCollection.IsBounded expectedBoundedness = + streaming ? PCollection.IsBounded.UNBOUNDED : PCollection.IsBounded.BOUNDED; + assertEquals(expectedBoundedness, output.isBounded()); + + PAssert.that(output).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); - List> expectedRecords = - Arrays.asList( - TestFixtures.FILE1SNAPSHOT1, - TestFixtures.FILE2SNAPSHOT1, - TestFixtures.FILE3SNAPSHOT1, - TestFixtures.FILE1SNAPSHOT2, - TestFixtures.FILE2SNAPSHOT2, - TestFixtures.FILE3SNAPSHOT2, - TestFixtures.FILE1SNAPSHOT3, - TestFixtures.FILE2SNAPSHOT3, - TestFixtures.FILE3SNAPSHOT3); 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()).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( - IcebergIO.readRows(catalogConfig) - .from(tableId) - .withTriggeringFrequency(Duration.standardSeconds(1)) - .toSnapshot(simpleTable.currentSnapshot().snapshotId())) - .apply(ParDo.of(new PrintRow())) - .setCoder(RowCoder.of(IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA))); - - assertEquals(PCollection.IsBounded.UNBOUNDED, output.isBounded()); - PAssert.that(output) - .satisfies( - (Iterable rows) -> { - assertThat(rows, containsInAnyOrder(expectedRows.toArray())); - return null; - }); + testPipeline.apply(readRows).apply(ParDo.of(new PrintRow())).setRowSchema(schema); + PCollection.IsBounded expectedBoundedness = + streaming ? PCollection.IsBounded.UNBOUNDED : PCollection.IsBounded.BOUNDED; + assertEquals(expectedBoundedness, output.isBounded()); + + PAssert.that(output).containsInAnyOrder(expectedRows); testPipeline.run(); } @@ -487,47 +554,16 @@ private static File createTempFile() throws IOException { return tempFile; } - private void commitData(Table simpleTable) throws IOException { - // first snapshot - 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(); - - // second snapshot - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file1s2.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT2)) - .appendFile( - warehouse.writeRecords( - "file2s2.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT2)) - .appendFile( - warehouse.writeRecords( - "file3s2.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT2)) - .commit(); + private IcebergCatalogConfig catalogConfig() { + Map catalogProps = + ImmutableMap.builder() + .put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP) + .put("warehouse", warehouse.location) + .build(); - // third snapshot - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file1s3.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT3)) - .appendFile( - warehouse.writeRecords( - "file2s3.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT3)) - .appendFile( - warehouse.writeRecords( - "file3s3.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT3)) - .commit(); + 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 f319abe5f902..8a461783e72b 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 @@ -22,10 +22,8 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; -import java.io.IOException; import java.util.Arrays; import java.util.HashMap; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.UUID; @@ -37,6 +35,7 @@ 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.Snapshot; import org.apache.iceberg.Table; @@ -84,9 +83,13 @@ public void testBuildTransformWithRow() { if (useIncrementalScan) { configBuilder = configBuilder - .withFieldValue("from_snapshot_exclusive", 123L) + .withFieldValue("from_snapshot", 123L) .withFieldValue("to_snapshot", 456L) - .withFieldValue("triggering_frequency_seconds", 789); + .withFieldValue("from_timestamp", 123L) + .withFieldValue("to_timestamp", 456L) + .withFieldValue("starting_strategy", "earliest") + .withFieldValue("streaming", true) + .withFieldValue("poll_interval_seconds", 789); } new IcebergReadSchemaTransformProvider().from(configBuilder.build()); @@ -100,7 +103,7 @@ public void testSimpleScan() throws Exception { Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); - commitData(simpleTable); + List> expectedRecords = warehouse.commitData(simpleTable); Map properties = new HashMap<>(); properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); @@ -112,26 +115,9 @@ public void testSimpleScan() throws Exception { .setCatalogName("name") .setCatalogProperties(properties); - List> expectedRecords = - Arrays.asList( - TestFixtures.FILE1SNAPSHOT1, - TestFixtures.FILE2SNAPSHOT1, - TestFixtures.FILE3SNAPSHOT1, - TestFixtures.FILE1SNAPSHOT2, - TestFixtures.FILE2SNAPSHOT2, - TestFixtures.FILE3SNAPSHOT2, - TestFixtures.FILE1SNAPSHOT3, - TestFixtures.FILE2SNAPSHOT3, - TestFixtures.FILE3SNAPSHOT3); if (useIncrementalScan) { - // only read files that were added in the second snapshot, - // ignoring the first and third snapshots. - expectedRecords = expectedRecords.subList(3, 6); - - Iterator snapshots = simpleTable.snapshots().iterator(); - long first = snapshots.next().snapshotId(); - long second = snapshots.next().snapshotId(); - readConfigBuilder = readConfigBuilder.setFromSnapshotExclusive(first).setToSnapshot(second); + readConfigBuilder = + readConfigBuilder.setToSnapshot(simpleTable.currentSnapshot().snapshotId()); } final List expectedRows = expectedRecords.stream() @@ -162,7 +148,7 @@ public void testReadUsingManagedTransform() throws Exception { Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); final Schema schema = IcebergUtils.icebergSchemaToBeamSchema(TestFixtures.SCHEMA); - commitData(simpleTable); + List> expectedRecords = warehouse.commitData(simpleTable); String yamlConfig = String.format( @@ -173,27 +159,16 @@ public void testReadUsingManagedTransform() throws Exception { + " warehouse: %s", identifier, CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, warehouse.location); - List> expectedRecords = - Arrays.asList( - TestFixtures.FILE1SNAPSHOT1, - TestFixtures.FILE2SNAPSHOT1, - TestFixtures.FILE3SNAPSHOT1, - TestFixtures.FILE1SNAPSHOT2, - TestFixtures.FILE2SNAPSHOT2, - TestFixtures.FILE3SNAPSHOT2, - TestFixtures.FILE1SNAPSHOT3, - TestFixtures.FILE2SNAPSHOT3, - TestFixtures.FILE3SNAPSHOT3); if (useIncrementalScan) { - // only read files that were added in the second snapshot, - // ignoring the first and third snapshots. - expectedRecords = expectedRecords.subList(3, 6); + // only read files that were added in the second and third snapshots, + // ignoring the first and fourth snapshots. + expectedRecords = expectedRecords.subList(3, 9); + + List snapshots = Lists.newArrayList(simpleTable.snapshots()); + long second = snapshots.get(1).snapshotId(); + long third = snapshots.get(2).snapshotId(); - Iterator snapshots = simpleTable.snapshots().iterator(); - long first = snapshots.next().snapshotId(); - long second = snapshots.next().snapshotId(); - yamlConfig += - String.format("\n" + "from_snapshot_exclusive: %s\n" + "to_snapshot: %s", first, second); + yamlConfig += String.format("\n" + "from_snapshot: %s\n" + "to_snapshot: %s", second, third); } final List expectedRows = expectedRecords.stream() @@ -216,48 +191,4 @@ public void testReadUsingManagedTransform() throws Exception { testPipeline.run(); } - - private void commitData(Table simpleTable) throws IOException { - // first snapshot - 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(); - - // second snapshot - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file1s2.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT2)) - .appendFile( - warehouse.writeRecords( - "file2s2.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT2)) - .appendFile( - warehouse.writeRecords( - "file3s2.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT2)) - .commit(); - - // third snapshot - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file1s3.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT3)) - .appendFile( - warehouse.writeRecords( - "file2s3.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT3)) - .appendFile( - warehouse.writeRecords( - "file3s3.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT3)) - .commit(); - } } 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..b99ac5d1bd6b 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 @@ -21,6 +21,7 @@ import java.io.File; import java.io.IOException; +import java.util.Arrays; import java.util.List; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.hadoop.conf.Configuration; @@ -170,4 +171,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/BigQueryMetastoreCatalogIT.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java index 3a8b47cb5a06..b9e0801df82b 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java @@ -119,9 +119,9 @@ public void testWriteToPartitionedAndValidateWithBQQuery() // Write with Beam Map config = managedIcebergConfig(tableId()); - PCollection input = pipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); + PCollection input = writePipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - pipeline.run().waitUntilFinish(); + writePipeline.run().waitUntilFinish(); // Fetch records using a BigQuery query and validate BigqueryClient bqClient = new BigqueryClient(getClass().getSimpleName()); 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 b55f98065bb2..424b243adf5a 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 @@ -21,6 +21,7 @@ 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; @@ -57,6 +58,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 +66,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; @@ -198,7 +201,8 @@ public void cleanUp() throws Exception { protected static final GcpOptions OPTIONS = TestPipeline.testingPipelineOptions().as(GcpOptions.class); private static final String RANDOM = UUID.randomUUID().toString(); - @Rule public TestPipeline pipeline = TestPipeline.create(); + @Rule public TestPipeline writePipeline = TestPipeline.create(); + @Rule public TestPipeline readPipeline = TestPipeline.create(); @Rule public TestName testName = new TestName(); @Rule public transient Timeout globalTimeout = Timeout.seconds(300); private static final int NUM_SHARDS = 10; @@ -388,10 +392,12 @@ public void testRead() throws Exception { Map config = managedIcebergConfig(tableId()); PCollection rows = - pipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); + writePipeline + .apply(Managed.read(Managed.ICEBERG).withConfig(config)) + .getSinglePCollection(); PAssert.that(rows).containsInAnyOrder(expectedRows); - pipeline.run().waitUntilFinish(); + writePipeline.run().waitUntilFinish(); } @Test @@ -401,35 +407,65 @@ public void testUnboundedRead() throws Exception { List expectedRows = populateTable(table); Map config = new HashMap<>(managedIcebergConfig(tableId())); - config.put("triggering_frequency_seconds", 30); + config.put("streaming", true); config.put("to_snapshot", table.currentSnapshot().snapshotId()); PCollection rows = - pipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); + writePipeline + .apply(Managed.read(Managed.ICEBERG).withConfig(config)) + .getSinglePCollection(); PAssert.that(rows).containsInAnyOrder(expectedRows); - pipeline.run().waitUntilFinish(); + writePipeline.run().waitUntilFinish(); } @Test - public void testReadSnapshotRange() throws Exception { - Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + public void testBatchReadBetweenSnapshots() throws Exception { + runReadBetween(true, false); + } - populateTable(table, "a"); - long from = table.currentSnapshot().snapshotId(); - List expectedRows = populateTable(table, "b"); - long to = table.currentSnapshot().snapshotId(); - populateTable(table, "c"); + @Test + public void testStreamingReadBetweenTimestamps() throws Exception { + runReadBetween(false, true); + } - Map config = new HashMap<>(managedIcebergConfig(tableId())); - config.put("from_snapshot_exclusive", from); - config.put("to_snapshot", to); + @Test + public void testWriteRead() { + Map config = managedIcebergConfig(tableId()); + PCollection input = writePipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); + input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); + writePipeline.run().waitUntilFinish(); - PCollection rows = - pipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); + PCollection output = + readPipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); + PAssert.that(output).containsInAnyOrder(inputRows); + } - PAssert.that(rows).containsInAnyOrder(expectedRows); - pipeline.run().waitUntilFinish(); + @Test + public void testWriteReadStreaming() { + int numRecords = numRecords(); + Map config = managedIcebergConfig(tableId()); + Map writeConfig = new HashMap<>(config); + writeConfig.put("triggering_frequency_seconds", 5); + PCollection input = + writePipeline + .apply(getStreamingSource()) + .apply( + MapElements.into(TypeDescriptors.rows()) + .via(instant -> ROW_FUNC.apply(instant.getMillis() % numRecords))) + .setRowSchema(BEAM_SCHEMA); + input.apply(Managed.write(Managed.ICEBERG).withConfig(writeConfig)); + writePipeline.run().waitUntilFinish(); + + Map readConfig = new HashMap<>(config); + readConfig.put("streaming", true); + readConfig.put("to_timestamp", System.currentTimeMillis()); + PCollection output = + readPipeline + .apply(Managed.read(Managed.ICEBERG).withConfig(readConfig)) + .getSinglePCollection(); + PAssert.that(output).containsInAnyOrder(inputRows); + readPipeline.run().waitUntilFinish(); } @Test @@ -437,9 +473,9 @@ 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); + PCollection input = writePipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - pipeline.run().waitUntilFinish(); + writePipeline.run().waitUntilFinish(); Table table = catalog.loadTable(TableIdentifier.parse(tableId())); assertTrue(table.schema().sameSchema(ICEBERG_SCHEMA)); @@ -465,9 +501,9 @@ public void testWriteToPartitionedTable() throws IOException { // Write with Beam Map config = managedIcebergConfig(tableId()); - PCollection input = pipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); + PCollection input = writePipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - pipeline.run().waitUntilFinish(); + writePipeline.run().waitUntilFinish(); // Read back and check records are correct List returnedRecords = readRecords(table); @@ -494,17 +530,17 @@ public void testStreamingWrite() throws IOException { // create elements from longs in range [0, 1000) PCollection input = - pipeline + writePipeline .apply(getStreamingSource()) .apply( MapElements.into(TypeDescriptors.rows()) .via(instant -> ROW_FUNC.apply(instant.getMillis() % numRecords))) .setRowSchema(BEAM_SCHEMA); - assertThat(input.isBounded(), equalTo(PCollection.IsBounded.UNBOUNDED)); + assertThat(input.isBounded(), equalTo(IsBounded.UNBOUNDED)); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - pipeline.run().waitUntilFinish(); + writePipeline.run().waitUntilFinish(); List returnedRecords = readRecords(table); assertThat( @@ -524,7 +560,7 @@ public void testStreamingWriteWithPriorWindowing() throws IOException { // over a span of 10 seconds, create elements from longs in range [0, 1000) PCollection input = - pipeline + writePipeline .apply(getStreamingSource()) .apply( Window.into(FixedWindows.of(Duration.standardSeconds(1))) @@ -534,10 +570,10 @@ 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(IsBounded.UNBOUNDED)); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - pipeline.run().waitUntilFinish(); + writePipeline.run().waitUntilFinish(); List returnedRecords = readRecords(table); assertThat( @@ -607,17 +643,17 @@ private void writeToDynamicDestinations( if (streaming) { writeConfig.put("triggering_frequency_seconds", 5); input = - pipeline + writePipeline .apply(getStreamingSource()) .apply( MapElements.into(TypeDescriptors.rows()) .via(instant -> ROW_FUNC.apply(instant.getMillis() % numRecords))); } else { - input = pipeline.apply(Create.of(inputRows)); + input = writePipeline.apply(Create.of(inputRows)); } input.setRowSchema(BEAM_SCHEMA).apply(Managed.write(Managed.ICEBERG).withConfig(writeConfig)); - pipeline.run().waitUntilFinish(); + writePipeline.run().waitUntilFinish(); Table table0 = catalog.loadTable(tableIdentifier0); Table table1 = catalog.loadTable(tableIdentifier1); @@ -678,4 +714,39 @@ 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); + } + + if (streaming) { + config.put("streaming", true); + } + + PCollection rows = + writePipeline + .apply(Managed.read(Managed.ICEBERG).withConfig(config)) + .getSinglePCollection(); + + IsBounded expectedBoundedness = streaming ? IsBounded.UNBOUNDED : IsBounded.BOUNDED; + assertEquals(expectedBoundedness, rows.isBounded()); + + PAssert.that(rows).containsInAnyOrder(expectedRows); + writePipeline.run().waitUntilFinish(); + } } From 297c309aba55bbf43f2f106ba4b63d23da35146c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 13 Feb 2025 12:56:25 -0500 Subject: [PATCH 12/32] trigger integration tests --- .github/trigger_files/IO_Iceberg_Integration_Tests.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 } From 5e3a2cc2da7834084854a357a6f8cbf2153daa9a Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Thu, 13 Feb 2025 13:47:23 -0500 Subject: [PATCH 13/32] small test fix --- .../apache/beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java | 1 + 1 file changed, 1 insertion(+) 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 424b243adf5a..6931a3bb445d 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 @@ -439,6 +439,7 @@ public void testWriteRead() { PCollection output = readPipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); PAssert.that(output).containsInAnyOrder(inputRows); + readPipeline.run().waitUntilFinish(); } @Test From 887eff1e2676b3ac3bb0a5b1e1480228a03d4668 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 25 Feb 2025 18:51:02 -0500 Subject: [PATCH 14/32] scan every snapshot individually; use snapshot commit timestamp to mark progress; convert GiB output iterable to list because of RunnerV2 bug --- .../sdk/io/iceberg/AppendFilesToTables.java | 2 +- .../sdk/io/iceberg/CreateReadTasksDoFn.java | 40 ++++++------ .../IcebergWriteSchemaTransformProvider.java | 5 +- .../sdk/io/iceberg/IncrementalScanSource.java | 36 ++++++---- .../sdk/io/iceberg/ReadFromGroupedTasks.java | 50 ++++++-------- .../beam/sdk/io/iceberg/ReadFromTasks.java | 19 +++--- .../apache/beam/sdk/io/iceberg/ReadTask.java | 9 --- .../sdk/io/iceberg/ReadTaskDescriptor.java | 4 ++ .../apache/beam/sdk/io/iceberg/ReadUtils.java | 36 +++++++++- .../beam/sdk/io/iceberg/SnapshotInfo.java | 58 ++++++++++++++--- .../beam/sdk/io/iceberg/SnapshotRange.java | 65 ------------------- .../sdk/io/iceberg/WatchForSnapshots.java | 56 ++++++++-------- .../beam/sdk/io/iceberg/ReadUtilsTest.java | 5 +- 13 files changed, 197 insertions(+), 188 deletions(-) delete mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java 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 index d3536b42c7d5..26a7fdec43bd 100644 --- 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + import java.io.IOException; import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.metrics.Counter; @@ -28,20 +30,21 @@ import org.apache.iceberg.IncrementalAppendScan; import org.apache.iceberg.ScanTaskParser; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; 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 {@link SnapshotRange}, and creates multiple {@link ReadTask}s. Each task - * represents a portion of a data file that was appended within the snapshot range. + * 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> { +class CreateReadTasksDoFn extends DoFn> { private static final Logger LOG = LoggerFactory.getLogger(CreateReadTasksDoFn.class); - private static final Counter numFileScanTasks = - Metrics.counter(CreateReadTasksDoFn.class, "numFileScanTasks"); + private static final Counter totalScanTasks = + Metrics.counter(CreateReadTasksDoFn.class, "totalScanTasks"); + private final IcebergCatalogConfig catalogConfig; CreateReadTasksDoFn(IcebergCatalogConfig catalogConfig) { @@ -50,18 +53,15 @@ class CreateReadTasksDoFn extends DoFn> out) + @Element SnapshotInfo snapshot, OutputReceiver> out) throws IOException, ExecutionException { - Table table = TableCache.get(range.getTableIdentifier(), catalogConfig.catalog()); - @Nullable Long fromSnapshot = range.getFromSnapshotExclusive(); - long toSnapshot = range.getToSnapshot(); + Table table = TableCache.get(snapshot.getTableIdentifier(), catalogConfig.catalog()); + @Nullable Long fromSnapshot = snapshot.getParentId(); + long toSnapshot = snapshot.getSnapshotId(); + System.out.println("xxx snapshot operation: " + snapshot.getOperation()); LOG.info("Planning to scan snapshot range ({}, {}]", fromSnapshot, toSnapshot); - IncrementalAppendScan scan = - table - .newIncrementalAppendScan() - .toSnapshot(toSnapshot) - .option(TableProperties.SPLIT_SIZE, String.valueOf(TableProperties.SPLIT_SIZE_DEFAULT)); + IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); if (fromSnapshot != null) { scan = scan.fromSnapshotExclusive(fromSnapshot); } @@ -73,16 +73,18 @@ public void process( for (FileScanTask fileScanTask : combinedScanTask.tasks()) { ReadTask task = ReadTask.builder() - .setTableIdentifierString(range.getTableIdentifierString()) .setFileScanTaskJson(ScanTaskParser.toJson(fileScanTask)) .setByteSize(fileScanTask.sizeBytes()) .build(); ReadTaskDescriptor descriptor = ReadTaskDescriptor.builder() - .setTableIdentifierString(range.getTableIdentifierString()) + .setTableIdentifierString(checkStateNotNull(snapshot.getTableIdentifierString())) + .setSnapshotTimestampMillis(snapshot.getTimestampMillis()) .build(); - out.output(KV.of(descriptor, task)); - numFileScanTasks.inc(); + + out.outputWithTimestamp( + KV.of(descriptor, task), Instant.ofEpochMilli(snapshot.getTimestampMillis())); + totalScanTasks.inc(); } } } 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 index 064abe1a0a9f..9ce210a126f4 100644 --- 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 @@ -19,14 +19,17 @@ 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.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.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.windowing.AfterPane; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; @@ -36,6 +39,7 @@ 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; @@ -47,6 +51,9 @@ * unbounded implementations) is used to process each task and output Beam rows. */ 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; @@ -78,15 +85,13 @@ private PCollection readUnbounded(PBegin input) { MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); return input .apply("Watch for Snapshots", new WatchForSnapshots(scanConfig, pollInterval)) + .setCoder(SnapshotInfo.getCoder()) .apply( "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) .setCoder(KvCoder.of(ReadTaskDescriptor.getCoder(), ReadTask.getCoder())) .apply( - "Apply User Trigger", Window.>into(new GlobalWindows()) - .triggering( - Repeatedly.forever( - AfterProcessingTime.pastFirstElementInPane().plusDelayOf(pollInterval))) + .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1))) .discardingFiredPanes()) .apply( GroupIntoBatches.ofByteSize( @@ -97,6 +102,17 @@ private PCollection readUnbounded(PBegin input) { KvCoder.of( ShardedKey.Coder.of(ReadTaskDescriptor.getCoder()), IterableCoder.of(ReadTask.getCoder()))) + .apply( + 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.getCatalogConfig()))); @@ -119,13 +135,9 @@ private PCollection readBounded(PBegin input, Table table) { ReadUtils.getToSnapshot(table, scanConfig), table.currentSnapshot().snapshotId()); return input .apply( - "Create Single Snapshot Range", - Create.of( - SnapshotRange.builder() - .setTableIdentifierString(scanConfig.getTableIdentifier()) - .setFromSnapshotExclusive(from) - .setToSnapshot(to) - .build())) + "Create Snapshot Ranges", + Create.of(ReadUtils.snapshotsBetween(table, scanConfig.getTableIdentifier(), from, to))) + .setCoder(SnapshotInfo.getCoder()) .apply( "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) .setCoder(KvCoder.of(ReadTaskDescriptor.getCoder(), ReadTask.getCoder())) 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 index 76af44ceff2a..a628edd9ecb2 100644 --- 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 @@ -18,26 +18,21 @@ package org.apache.beam.sdk.io.iceberg; import java.io.IOException; -import java.util.Iterator; 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.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; -import org.apache.beam.sdk.util.ShardedKey; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; -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.FileScanTask; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.mapping.NameMapping; -import org.apache.iceberg.mapping.NameMappingParser; -import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; /** * Unbounded read implementation. @@ -49,9 +44,10 @@ * equals the batch size. */ @DoFn.BoundedPerElement -class ReadFromGroupedTasks - extends DoFn, Iterable>, Row> { +class ReadFromGroupedTasks extends DoFn>, Row> { private final IcebergCatalogConfig catalogConfig; + private final Counter scanTasksCompleted = + Metrics.counter(ReadFromGroupedTasks.class, "scanTasksCompleted"); ReadFromGroupedTasks(IcebergCatalogConfig catalogConfig) { this.catalogConfig = catalogConfig; @@ -59,17 +55,15 @@ class ReadFromGroupedTasks @ProcessElement public void process( - @Element KV, Iterable> element, + @Element KV> element, RestrictionTracker tracker, OutputReceiver out) throws IOException, ExecutionException { - String tableIdentifier = element.getKey().getKey().getTableIdentifierString(); - List readTasks = Lists.newArrayList(element.getValue()); + String tableIdentifier = element.getKey().getTableIdentifierString(); + Instant timestamp = Instant.ofEpochMilli(element.getKey().getSnapshotTimestampMillis()); + List readTasks = element.getValue(); Table table = TableCache.get(tableIdentifier, catalogConfig.catalog()); Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); - @Nullable String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); - NameMapping mapping = - nameMapping != null ? NameMappingParser.fromJson(nameMapping) : NameMapping.empty(); // SDF can split by the number of read tasks for (long taskIndex = tracker.currentRestriction().getFrom(); @@ -79,33 +73,29 @@ public void process( return; } FileScanTask task = readTasks.get((int) taskIndex).getFileScanTask(); - try (CloseableIterable reader = ReadUtils.createReader(task, table, mapping)) { + try (CloseableIterable reader = ReadUtils.createReader(task, table)) { for (Record record : reader) { Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); - out.output(row); + out.outputWithTimestamp(row, timestamp); } } + scanTasksCompleted.inc(); } } @GetInitialRestriction - public OffsetRange getInitialRange( - @Element KV, Iterable> element) { - return new OffsetRange(0, Iterables.size(element.getValue())); + public OffsetRange getInitialRange(@Element KV> element) { + return new OffsetRange(0, element.getValue().size()); } @GetSize public double getSize( - @Element KV, Iterable> element, - @Restriction OffsetRange restriction) - throws Exception { + @Element KV> element, + @Restriction OffsetRange restriction) { double size = 0; - Iterator iterator = element.getValue().iterator(); - for (long i = 0; i < restriction.getTo() && iterator.hasNext(); i++) { - ReadTask task = iterator.next(); - if (i >= restriction.getFrom()) { - size += task.getByteSize(); - } + 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 index f2d751d0616c..2de86bfd4771 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -19,26 +19,28 @@ 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.schemas.Schema; 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.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.mapping.NameMapping; -import org.apache.iceberg.mapping.NameMappingParser; -import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; /** * Bounded read implementation. * *

For each {@link ReadTask}, reads Iceberg {@link Record}s, and converts to Beam {@link Row}s. */ +// @BoundedPerElement class ReadFromTasks extends DoFn, Row> { private final IcebergCatalogConfig catalogConfig; + private final Counter scanTasksCompleted = + Metrics.counter(ReadFromTasks.class, "scanTasksCompleted"); ReadFromTasks(IcebergCatalogConfig catalogConfig) { this.catalogConfig = catalogConfig; @@ -48,20 +50,19 @@ class ReadFromTasks extends DoFn, Row> { public void process(@Element KV element, OutputReceiver out) throws IOException, ExecutionException { String tableIdentifier = element.getKey().getTableIdentifierString(); + Instant timestamp = Instant.ofEpochMilli(element.getKey().getSnapshotTimestampMillis()); ReadTask readTask = element.getValue(); Table table = TableCache.get(tableIdentifier, catalogConfig.catalog()); Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); - @Nullable String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); - NameMapping mapping = - nameMapping != null ? NameMappingParser.fromJson(nameMapping) : NameMapping.empty(); FileScanTask task = readTask.getFileScanTask(); - try (CloseableIterable reader = ReadUtils.createReader(task, table, mapping)) { + try (CloseableIterable reader = ReadUtils.createReader(task, table)) { for (Record record : reader) { Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); - out.output(row); + out.outputWithTimestamp(row, timestamp); } } + 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 index 220b46b9fdb7..940fcd9adb65 100644 --- 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 @@ -50,8 +50,6 @@ static Builder builder() { return new AutoValue_ReadTask.Builder(); } - abstract String getTableIdentifierString(); - abstract String getFileScanTaskJson(); abstract long getByteSize(); @@ -66,17 +64,10 @@ FileScanTask getFileScanTask() { @AutoValue.Builder abstract static class Builder { - abstract Builder setTableIdentifierString(String table); - abstract Builder setFileScanTaskJson(String jsonTask); abstract Builder setByteSize(long size); - @SchemaIgnore - Builder setFileScanTask(FileScanTask task) { - return setFileScanTaskJson(ScanTaskParser.toJson(task)); - } - 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 index 2e5c37fdcc19..ce30e96bf58c 100644 --- 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 @@ -48,10 +48,14 @@ static Builder builder() { abstract String getTableIdentifierString(); + abstract long getSnapshotTimestampMillis(); + @AutoValue.Builder abstract static class Builder { abstract Builder setTableIdentifierString(String table); + abstract Builder setSnapshotTimestampMillis(long millis); + 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 index 162f22262f2f..037d6be0265f 100644 --- 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 @@ -17,18 +17,24 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.iceberg.util.SnapshotUtil.ancestorsOf; + import java.util.Collection; import java.util.Collections; +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.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.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.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; @@ -38,6 +44,7 @@ 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; @@ -47,7 +54,7 @@ import org.apache.parquet.ParquetReadOptions; import org.checkerframework.checker.nullness.qual.Nullable; -class ReadUtils { +public class ReadUtils { // default is 8MB. keep this low to avoid overwhelming memory private static final int MAX_FILE_BUFFER_SIZE = 1 << 20; // 1MB private static final Collection READ_PROPERTIES_TO_REMOVE = @@ -57,7 +64,7 @@ class ReadUtils { "parquet.read.support.class", "parquet.crypto.factory.class"); - static ParquetReader createReader(FileScanTask task, Table table, NameMapping mapping) { + public static ParquetReader createReader(FileScanTask task, Table table) { String filePath = task.file().path().toString(); InputFile inputFile; try (FileIO io = table.io()) { @@ -84,6 +91,10 @@ static ParquetReader createReader(FileScanTask task, Table table, NameMa .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(), @@ -150,4 +161,25 @@ static ParquetReader createReader(FileScanTask task, Table table, NameMa 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)) + .collect(Collectors.toList()); + Collections.reverse(snapshotIds); + return snapshotIds; + } } 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..4c67066b1926 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,7 +17,10 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + import com.google.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -26,8 +29,11 @@ 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.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/SnapshotRange.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java deleted file mode 100644 index 4c49f53043e8..000000000000 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/SnapshotRange.java +++ /dev/null @@ -1,65 +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.io.Serializable; -import org.apache.beam.sdk.schemas.AutoValueSchema; -import org.apache.beam.sdk.schemas.annotations.DefaultSchema; -import org.apache.beam.sdk.schemas.annotations.SchemaIgnore; -import org.apache.iceberg.catalog.TableIdentifier; -import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -import org.checkerframework.checker.nullness.qual.Nullable; - -@DefaultSchema(AutoValueSchema.class) -@AutoValue -abstract class SnapshotRange implements Serializable { - private transient @MonotonicNonNull TableIdentifier cachedTableIdentifier; - - static Builder builder() { - return new AutoValue_SnapshotRange.Builder(); - } - - abstract String getTableIdentifierString(); - - /** - * Snapshot to start reading from (exclusive). If null, starts reading from the oldest snapshot. - */ - abstract @Nullable Long getFromSnapshotExclusive(); - - abstract long getToSnapshot(); - - @SchemaIgnore - public TableIdentifier getTableIdentifier() { - if (cachedTableIdentifier == null) { - cachedTableIdentifier = TableIdentifier.parse(getTableIdentifierString()); - } - return cachedTableIdentifier; - } - - @AutoValue.Builder - abstract static class Builder { - abstract Builder setTableIdentifierString(String table); - - abstract Builder setFromSnapshotExclusive(@Nullable Long fromSnapshot); - - abstract Builder setToSnapshot(Long toSnapshot); - - abstract SnapshotRange build(); - } -} 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 index d6679a156025..17a337afd03c 100644 --- 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 @@ -21,6 +21,7 @@ import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; import org.apache.beam.sdk.metrics.Gauge; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.Create; @@ -46,7 +47,7 @@ * *

A downstream transform will create a list of read tasks for each range. */ -class WatchForSnapshots extends PTransform> { +class WatchForSnapshots extends PTransform> { private final Duration pollInterval; private final IcebergScanConfig scanConfig; @@ -56,18 +57,19 @@ class WatchForSnapshots extends PTransform> { } @Override - public PCollection expand(PBegin input) { + public PCollection expand(PBegin input) { return input .apply(Create.of(scanConfig.getTableIdentifier())) .apply( "Watch for Snapshots", - Watch.growthOf(new SnapshotPollFn(scanConfig)).withPollInterval(pollInterval)) + Watch.growthOf(new SnapshotPollFn(scanConfig)) + .withPollInterval(pollInterval) + .withOutputCoder(SnapshotInfo.getCoder())) .apply( - "Strip key", - MapElements.into(TypeDescriptor.of(SnapshotRange.class)).via(KV::getValue)); + "Strip key", MapElements.into(TypeDescriptor.of(SnapshotInfo.class)).via(KV::getValue)); } - private static class SnapshotPollFn extends Watch.Growth.PollFn { + private static class SnapshotPollFn extends Watch.Growth.PollFn { private final Gauge latestSnapshot = Metrics.gauge(SnapshotPollFn.class, "latestSnapshot"); private final IcebergScanConfig scanConfig; private @Nullable Long fromSnapshotId; @@ -77,7 +79,7 @@ private static class SnapshotPollFn extends Watch.Growth.PollFn apply(String tableIdentifier, Context c) { + public PollResult apply(String tableIdentifier, Context c) { // fetch a fresh table to catch updated snapshots Table table = TableCache.getRefreshed(tableIdentifier, scanConfig.getCatalogConfig().catalog()); @@ -86,39 +88,35 @@ public PollResult apply(String tableIdentifier, Context c) { if (fromSnapshotId == null) { fromSnapshotId = ReadUtils.getFromSnapshotExclusive(table, scanConfig); } - Instant timestamp = Instant.now(); Snapshot currentSnapshot = table.currentSnapshot(); if (currentSnapshot == null || Objects.equal(currentSnapshot.snapshotId(), fromSnapshotId)) { - // no new snapshot since last poll. return empty result. - return getPollResult(null, timestamp, isComplete); + // no new snapshots since last poll. return empty result. + return getPollResult(null, isComplete); } Long currentSnapshotId = currentSnapshot.snapshotId(); - // if no upper bound is specified, we read up to the current snapshot - Long toSnapshot = MoreObjects.firstNonNull(userSpecifiedToSnapshot, currentSnapshotId); - latestSnapshot.set(toSnapshot); + // if no upper bound is specified, we poll up to the current snapshot + long toSnapshotId = MoreObjects.firstNonNull(userSpecifiedToSnapshot, currentSnapshotId); + latestSnapshot.set(toSnapshotId); - SnapshotRange range = - SnapshotRange.builder() - .setFromSnapshotExclusive(fromSnapshotId) - .setToSnapshot(toSnapshot) - .setTableIdentifierString(tableIdentifier) - .build(); - - // update lower bound to current snapshot - fromSnapshotId = currentSnapshotId; - - return getPollResult(range, timestamp, isComplete); + List snapshots = + ReadUtils.snapshotsBetween(table, tableIdentifier, fromSnapshotId, toSnapshotId); + return getPollResult(snapshots, isComplete); } /** Returns an appropriate PollResult based on the requested boundedness. */ - private PollResult getPollResult( - @Nullable SnapshotRange range, Instant timestamp, boolean isComplete) { - List> timestampedValues = - range == null + private PollResult getPollResult( + @Nullable List snapshots, boolean isComplete) { + List> timestampedValues = + snapshots == null ? Collections.emptyList() - : Collections.singletonList(TimestampedValue.of(range, timestamp)); + : snapshots.stream() + .map( + snapshot -> + TimestampedValue.of( + snapshot, Instant.ofEpochMilli(snapshot.getTimestampMillis()))) + .collect(Collectors.toList()); return isComplete ? PollResult.complete(timestampedValues) // stop at specified snapshot 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 index 768d6865b759..a2002b1e3a5d 100644 --- 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 @@ -71,9 +71,10 @@ public void testCreateReader() throws IOException { for (CombinedScanTask combinedScanTask : iterable) { for (FileScanTask fileScanTask : combinedScanTask.tasks()) { String fileName = Iterables.getLast(Splitter.on("/").split(fileScanTask.file().path())); - ParquetReader reader = ReadUtils.createReader(fileScanTask, simpleTable, null); List recordsRead = new ArrayList<>(); - reader.forEach(recordsRead::add); + try (ParquetReader reader = ReadUtils.createReader(fileScanTask, simpleTable)) { + reader.forEach(recordsRead::add); + } assertEquals(data.get(fileName), recordsRead); numFiles++; From 6cfc2d86996ed4b0d96e1b595e8f6b6205f8e173 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 3 Mar 2025 14:35:19 -0500 Subject: [PATCH 15/32] new schematransform for cdc streaming; add watermark configs --- .../pipeline/v1/external_transforms.proto | 2 + sdks/java/io/iceberg/build.gradle | 2 +- .../sdk/io/iceberg/CreateReadTasksDoFn.java | 68 ++++-- ...IcebergCdcReadSchemaTransformProvider.java | 223 ++++++++++++++++++ .../apache/beam/sdk/io/iceberg/IcebergIO.java | 154 +++++++++--- .../IcebergReadSchemaTransformProvider.java | 22 +- .../sdk/io/iceberg/IcebergScanConfig.java | 115 ++++++++- .../IcebergSchemaTransformTranslation.java | 15 ++ .../sdk/io/iceberg/IncrementalScanSource.java | 47 ++-- .../sdk/io/iceberg/ReadFromGroupedTasks.java | 37 ++- .../beam/sdk/io/iceberg/ReadFromTasks.java | 20 +- .../apache/beam/sdk/io/iceberg/ReadTask.java | 9 + .../sdk/io/iceberg/ReadTaskDescriptor.java | 4 - .../apache/beam/sdk/io/iceberg/ReadUtils.java | 119 +++++++++- .../sdk/io/iceberg/WatchForSnapshots.java | 39 ++- ...ergCdcReadSchemaTransformProviderTest.java | 173 ++++++++++++++ .../sdk/io/iceberg/IcebergIOReadTest.java | 186 ++++++++++++++- ...cebergReadSchemaTransformProviderTest.java | 60 +---- ...IcebergSchemaTransformTranslationTest.java | 149 +++++++++--- .../beam/sdk/io/iceberg/ReadUtilsTest.java | 42 ++++ .../catalog/BigQueryMetastoreCatalogIT.java | 99 +++++++- .../iceberg/catalog/IcebergCatalogBaseIT.java | 150 +++++++----- .../org/apache/beam/sdk/managed/Managed.java | 2 + 23 files changed, 1446 insertions(+), 291 deletions(-) create mode 100644 sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java create mode 100644 sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProviderTest.java 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 ab2ef43e5278..32c95ea8260c 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -89,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/CreateReadTasksDoFn.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/CreateReadTasksDoFn.java index 26a7fdec43bd..ad7b65d4e05b 100644 --- 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 @@ -20,19 +20,21 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import java.io.IOException; +import java.util.Collections; +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; @@ -40,50 +42,78 @@ * 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> { +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 IcebergCatalogConfig catalogConfig; + private final IcebergScanConfig scanConfig; + private final @Nullable String watermarkColumnName; - CreateReadTasksDoFn(IcebergCatalogConfig catalogConfig) { - this.catalogConfig = catalogConfig; + CreateReadTasksDoFn(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; + this.watermarkColumnName = scanConfig.getWatermarkColumn(); } @ProcessElement public void process( - @Element SnapshotInfo snapshot, OutputReceiver> out) + @Element KV> element, + OutputReceiver> out) throws IOException, ExecutionException { - Table table = TableCache.get(snapshot.getTableIdentifier(), catalogConfig.catalog()); - @Nullable Long fromSnapshot = snapshot.getParentId(); - long toSnapshot = snapshot.getSnapshotId(); - System.out.println("xxx snapshot operation: " + snapshot.getOperation()); + Table table = TableCache.get(element.getKey(), scanConfig.getCatalogConfig().catalog()); + List snapshots = element.getValue(); - LOG.info("Planning to scan snapshot range ({}, {}]", fromSnapshot, toSnapshot); - IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); - if (fromSnapshot != null) { - scan = scan.fromSnapshotExclusive(fromSnapshot); + // 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 id range ({}, {}]", fromSnapshot, toSnapshot); + IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); + if (fromSnapshot != null) { + scan = scan.fromSnapshotExclusive(fromSnapshot); + } + if (watermarkColumnName != null) { + scan = scan.includeColumnStats(Collections.singletonList(watermarkColumnName)); + } + + createAndOutputReadTasks(scan, snapshot, out); } + } + private void createAndOutputReadTasks( + IncrementalAppendScan scan, + SnapshotInfo snapshot, + OutputReceiver> out) + throws IOException { try (CloseableIterable combinedScanTasks = scan.planTasks()) { for (CombinedScanTask combinedScanTask : combinedScanTasks) { // A single DataFile can be broken up into multiple FileScanTasks - // if it is large enough. for (FileScanTask fileScanTask : combinedScanTask.tasks()) { ReadTask task = ReadTask.builder() .setFileScanTaskJson(ScanTaskParser.toJson(fileScanTask)) - .setByteSize(fileScanTask.sizeBytes()) + .setByteSize(fileScanTask.file().fileSizeInBytes()) + .setOperation(snapshot.getOperation()) + .setSnapshotTimestampMillis(snapshot.getTimestampMillis()) .build(); ReadTaskDescriptor descriptor = ReadTaskDescriptor.builder() .setTableIdentifierString(checkStateNotNull(snapshot.getTableIdentifierString())) - .setSnapshotTimestampMillis(snapshot.getTimestampMillis()) .build(); - out.outputWithTimestamp( - KV.of(descriptor, task), Instant.ofEpochMilli(snapshot.getTimestampMillis())); + out.output(KV.of(descriptor, task)); totalScanTasks.inc(); } } 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..c5f412cfcdd9 --- /dev/null +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java @@ -0,0 +1,223 @@ +/* + * 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()) + .from(TableIdentifier.parse(configuration.getTable())) + .fromSnapshot(configuration.getFromSnapshot()) + .toSnapshot(configuration.getToSnapshot()) + .fromTimestamp(configuration.getFromTimestamp()) + .toTimestamp(configuration.getToTimestamp()) + .withStartingStrategy(strategy) + .withWatermarkColumn(configuration.getWatermarkColumn()) + .withWatermarkTimeUnit(configuration.getWatermarkTimeUnit()) + .streaming(true); + + @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( + "The interval at which to poll for new snapshots. Defaults to 60 seconds.") + abstract @Nullable Integer getPollIntervalSeconds(); + + @SchemaFieldDescription( + "The column used to derive event time for tracking progress. Uses the snapshot's commit timestamp by default.") + abstract @Nullable String getWatermarkColumn(); + + @SchemaFieldDescription( + "Use only when the watermark column is set to a Long type. Specifies the TimeUnit represented by the watermark column. Default is 'microseconds'. " + + "Check https://docs.oracle.com/javase/8/docs/api///?java/util/concurrent/TimeUnit.html for possible values.") + abstract @Nullable String getWatermarkTimeUnit(); + + @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 setWatermarkColumn(String column); + + abstract Builder setWatermarkTimeUnit(String watermarkTimeUnit); + + 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 46a1300acbe1..1151618fa2f2 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 @@ -22,6 +22,7 @@ import com.google.auto.value.AutoValue; import java.util.Arrays; import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.schemas.Schema; @@ -69,6 +70,13 @@ * .apply(Managed.read(ICEBERG).withConfig(config)) * .getSinglePCollection() * .apply(ParDo.of(...)); + * + * + * ====== READ CDC ====== + * pipeline + * .apply(Managed.read(ICEBERG_CDC).withConfig(config)) + * .getSinglePCollection() + * .apply(ParDo.of(...)); * }

* * Look for more detailed examples below. @@ -157,20 +165,6 @@ * * * - * {@code streaming} - * {@code boolean} - * Enables streaming reads. By default, the streaming source will start reading from the latest - * snapshot (inclusive) and continue polling forever based on the specified {@code poll_interval_seconds}. - * - * - * - * {@code poll_interval_seconds} - * {@code int} - * - * The interval at which to scan the table for new snapshots. Defaults to 60 seconds. For streaming reads only. - * - * - * * {@code starting_strategy} * {@code str} * @@ -182,6 +176,43 @@ *

Defaults to {@code earliest} for batch, and {@code latest} for streaming. * * + * + * {@code watermark_column} + * {@code str} + * + * The column used to derive event time to track progress. Must be of type: + *

    + *
  • {@code timestamp}
  • + *
  • {@code timestamptz}
  • + *
  • {@code long} (micros)
  • + *
+ * + * + * + * {@code watermark_time_unit} + * {@code str} + * + * Use only when {@code watermark_column} is set to a column of type Long. Specifies the TimeUnit represented by the watermark column. + * Default is {@code "microseconds"}. + * + *

Check {@link TimeUnit} for possible values. + * + * + * + * + *

CDC Streaming Source options

+ * + * + * + * + * + * + * + * + * + * *
Parameter Type Description
{@code poll_interval_seconds} {@code int} + * The interval at which to scan the table for new snapshots. Defaults to 60 seconds. + *
* *

Beam Rows

@@ -381,7 +412,7 @@ * *

Reading from Tables

* - * A simple batch read from an Iceberg table looks like this: + * With the following configuration, * *
{@code
  * Map config = Map.of(
@@ -389,21 +420,47 @@
  *         "catalog_name", name,
  *         "catalog_properties", Map.of(...),
  *         "config_properties", Map.of(...));
+ * }
+ * + * Example of a simple batch read: * + *
{@code
  * PCollection = pipeline
  *     .apply(Managed.read(ICEBERG).withConfig(config))
  *     .getSinglePCollection();
  * }
* - * Setting {@code streaming} to {@code true} enables streaming read mode, which continuously - * polls the table for new snapshots. The default polling interval is 60 seconds, but this can be - * overridden using {@code poll_interval_seconds}: + * Example of a simple CDC streaming read: + * + *
{@code
+ * PCollection = pipeline
+ *     .apply(Managed.read(ICEBERG_CDC).withConfig(config))
+ *     .getSinglePCollection()
+ *     .apply(ReadUtils.extractRecords());
+ * }
+ * + *

The streaming source continuously polls the table for new append-only snapshots, with a default interval + * of 60 seconds. This can be overridden using {@code poll_interval_seconds}. + *

Note: Full CDC is not supported yet. * *

{@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: + * + *

    + *
  • {@code "record"}: a Row representing the data record + *
  • {@code "operation"}: the snapshot operation associated with this record (e.g. + * "append", "replace", "delete") + *
+ * *

Choosing a Starting Point

* * By default, a batch read will start reading from the earliest (oldest) table snapshot. A @@ -460,7 +517,25 @@ * .getSinglePCollection(); * } * - * Note: An end point can also be set when performing a streaming read. + * Note: An end point can also be set when performing a streaming read. + * + *

Handling Watermarks

+ * + * By default, a snapshot's commit timestamp is assigned to all the records it contains. + * + *

For greater watermark precision, specify a {@code watermark_column}, which allows the + * source to extract the lower bound value from each data file and assign it to the corresponding + * records. For Long column types, you can also specify the TimeUnit represented by the column: + * + *

{@code
+ * config.put("watermark_column", "flight_arrival");
+ * config.put("watermark_time_unit", "hours");
+ * }
+ * + * Note: this requires the data files to have been written with metrics enabled. For more + * details, refer to the write + * properties. */ @Internal public class IcebergIO { @@ -567,11 +642,15 @@ public enum StartingStrategy { abstract @Nullable Long getToTimestamp(); - abstract @Nullable Duration getPollInterval(); + abstract @Nullable StartingStrategy getStartingStrategy(); abstract @Nullable Boolean getStreaming(); - abstract @Nullable StartingStrategy getStartingStrategy(); + abstract @Nullable Duration getPollInterval(); + + abstract @Nullable String getWatermarkColumn(); + + abstract @Nullable String getWatermarkTimeUnit(); abstract Builder toBuilder(); @@ -589,11 +668,15 @@ abstract static class Builder { abstract Builder setToTimestamp(@Nullable Long toTimestamp); - abstract Builder setPollInterval(@Nullable Duration triggeringFrequency); + abstract Builder setStartingStrategy(@Nullable StartingStrategy strategy); abstract Builder setStreaming(@Nullable Boolean streaming); - abstract Builder setStartingStrategy(@Nullable StartingStrategy strategy); + abstract Builder setPollInterval(@Nullable Duration triggeringFrequency); + + abstract Builder setWatermarkColumn(@Nullable String column); + + abstract Builder setWatermarkTimeUnit(@Nullable String timeUnit); abstract ReadRows build(); } @@ -630,6 +713,14 @@ public ReadRows withStartingStrategy(@Nullable StartingStrategy strategy) { return toBuilder().setStartingStrategy(strategy).build(); } + public ReadRows withWatermarkColumn(@Nullable String column) { + return toBuilder().setWatermarkColumn(column).build(); + } + + public ReadRows withWatermarkTimeUnit(@Nullable String timeUnit) { + return toBuilder().setWatermarkTimeUnit(timeUnit).build(); + } + @Override public PCollection expand(PBegin input) { TableIdentifier tableId = @@ -650,16 +741,17 @@ public PCollection expand(PBegin input) { .setStartingStrategy(getStartingStrategy()) .setStreaming(getStreaming()) .setPollInterval(getPollInterval()) + .setWatermarkColumn(getWatermarkColumn()) + .setWatermarkTimeUnit(getWatermarkTimeUnit()) .build(); - scanConfig.validate(); + scanConfig.validate(table); - if (scanConfig.useIncrementalSource()) { - return input - .apply(new IncrementalScanSource(scanConfig)) - .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); - } + PTransform> source = + scanConfig.useIncrementalSource() + ? new IncrementalScanSource(scanConfig) + : Read.from(new ScanSource(scanConfig)); - return input.apply(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 aa92e7af86e4..13febe902a9f 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 @@ -43,7 +43,6 @@ 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 @@ -114,12 +113,8 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .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)); - } + .withWatermarkColumn(configuration.getWatermarkColumn()) + .withWatermarkTimeUnit(configuration.getWatermarkTimeUnit()); PCollection output = input.getPipeline().apply(readRows); @@ -177,6 +172,15 @@ static Builder builder() { + "by setting a starting snapshot or timestamp. Defaults to earliest for batch, and latest for streaming.") abstract @Nullable String getStartingStrategy(); + @SchemaFieldDescription( + "The column used to derive event time for tracking progress. Uses the snapshot's commit timestamp by default.") + abstract @Nullable String getWatermarkColumn(); + + @SchemaFieldDescription( + "Use only when the watermark column is set to a Long type. Specifies the TimeUnit represented by the watermark column. Default is 'microseconds'. " + + "Check https://docs.oracle.com/javase/8/docs/api///?java/util/concurrent/TimeUnit.html for possible values.") + abstract @Nullable String getWatermarkTimeUnit(); + @AutoValue.Builder abstract static class Builder { abstract Builder setTable(String table); @@ -201,6 +205,10 @@ abstract static class Builder { abstract Builder setStartingStrategy(String strategy); + abstract Builder setWatermarkColumn(String column); + + abstract Builder setWatermarkTimeUnit(String watermarkTimeUnit); + abstract Configuration 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 7e1d3f98e825..794deb933275 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,16 +17,31 @@ */ package org.apache.beam.sdk.io.iceberg; +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; + import com.google.auto.value.AutoValue; import java.io.Serializable; +import java.util.Arrays; +import java.util.Set; +import java.util.concurrent.TimeUnit; +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.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.ImmutableMap; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.MetricsModes; +import org.apache.iceberg.MetricsModes.Full; +import org.apache.iceberg.MetricsModes.Truncate; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; @@ -36,10 +51,12 @@ public abstract class IcebergScanConfig implements Serializable { public boolean useIncrementalSource() { return MoreObjects.firstNonNull(getStreaming(), false) + || getPollInterval() != null || getToTimestamp() != null || getFromSnapshotInclusive() != null || getToSnapshot() != null - || getStartingStrategy() != null; + || getStartingStrategy() != null + || getWatermarkColumn() != null; } private transient @MonotonicNonNull Table cachedTable; @@ -49,6 +66,10 @@ public enum ScanType { BATCH } + private final Set WATERMARK_COLUMN_TYPES = + ImmutableSet.of( + Types.LongType.get(), Types.TimestampType.withoutZone(), Types.TimestampType.withZone()); + @Pure public abstract ScanType getScanType(); @@ -109,6 +130,9 @@ public Table getTable() { @Pure public abstract @Nullable Long getToTimestamp(); + @Pure + public abstract @Nullable StartingStrategy getStartingStrategy(); + @Pure public abstract @Nullable Boolean getStreaming(); @@ -116,7 +140,10 @@ public Table getTable() { public abstract @Nullable Duration getPollInterval(); @Pure - public abstract @Nullable StartingStrategy getStartingStrategy(); + public abstract @Nullable String getWatermarkColumn(); + + @Pure + public abstract @Nullable String getWatermarkTimeUnit(); @Pure public abstract @Nullable String getTag(); @@ -192,11 +219,15 @@ public Builder setTableIdentifier(String... names) { public abstract Builder setToTimestamp(@Nullable Long timestamp); + public abstract Builder setStartingStrategy(@Nullable StartingStrategy strategy); + public abstract Builder setStreaming(@Nullable Boolean streaming); public abstract Builder setPollInterval(@Nullable Duration pollInterval); - public abstract Builder setStartingStrategy(@Nullable StartingStrategy strategy); + public abstract Builder setWatermarkColumn(@Nullable String column); + + public abstract Builder setWatermarkTimeUnit(@Nullable String timeUnit); public abstract Builder setTag(@Nullable String tag); @@ -205,23 +236,81 @@ public Builder setTableIdentifier(String... names) { public abstract IcebergScanConfig build(); } - void validate() { + void validate(Table table) { if (getStartingStrategy() != null) { - Preconditions.checkArgument( + checkArgument( getFromTimestamp() == null && getFromSnapshotInclusive() == null, - "Invalid source configuration: 'from_timestamp' and 'from_snapshot' are not allowed when 'starting_strategy' is set"); + error( + "'from_timestamp' and 'from_snapshot' are not allowed when 'starting_strategy' is set")); } - Preconditions.checkArgument( + checkArgument( getFromTimestamp() == null || getFromSnapshotInclusive() == null, - "Invalid source configuration: Only one of 'from_timestamp' or 'from_snapshot' can be set"); - Preconditions.checkArgument( + error("only one of 'from_timestamp' or 'from_snapshot' can be set")); + checkArgument( getToTimestamp() == null || getToSnapshot() == null, - "Invalid source configuration: Only one of 'to_timestamp' or 'to_snapshot' can be set"); + error("only one of 'to_timestamp' or 'to_snapshot' can be set")); if (getPollInterval() != null) { - Preconditions.checkArgument( + checkArgument( Boolean.TRUE.equals(getStreaming()), - "Invalid source configuration: 'poll_interval_seconds' can only be set when streaming is true"); + error("'poll_interval_seconds' can only be set when streaming is true")); } + + @Nullable String watermarkColumn = getWatermarkColumn(); + if (watermarkColumn != null) { + org.apache.iceberg.Schema icebergSchema = IcebergUtils.beamSchemaToIcebergSchema(getSchema()); + NestedField field = + checkArgumentNotNull( + icebergSchema.findField(watermarkColumn), + error("the specified 'watermark_column' field does not exist: '%s'."), + watermarkColumn); + + Type type = field.type(); + checkArgument( + WATERMARK_COLUMN_TYPES.contains(type), + error("invalid 'watermark_column' type: %s. Valid types are %s."), + type, + WATERMARK_COLUMN_TYPES); + + MetricsModes.MetricsMode mode = MetricsConfig.forTable(table).columnMode(watermarkColumn); + checkState( + mode instanceof Truncate || mode instanceof Full, + error( + "source table '%s' is not configured to capture lower bound metrics for the specified watermark column '%s'. " + + "Valid metric modes are '%s', but found '%s'. See " + + "table option 'write.metadata.metrics...'in " + + "https://iceberg.apache.org/docs/latest/configuration/#write-properties for more information."), + getTableIdentifier(), + watermarkColumn, + ImmutableSet.of("truncate(n)", "full"), + mode); + + @Nullable String watermarkTimeUnit = getWatermarkTimeUnit(); + if (watermarkTimeUnit != null) { + checkArgument( + type.equals(Types.LongType.get()), + error( + "'watermark_time_unit' is only applicable for Long types. The specified " + + "'watermark_column' is of type: %s"), + type); + + Set validTimeUnits = + Arrays.stream(TimeUnit.values()).map(Enum::name).collect(Collectors.toSet()); + checkArgument( + validTimeUnits.contains(watermarkTimeUnit.toUpperCase()), + error("invalid 'watermark_time_unit': %s. Please choose one of: %s"), + watermarkTimeUnit, + validTimeUnits); + } + } else { + checkArgument( + getWatermarkTimeUnit() == null, + error( + "cannot set 'watermark_time_unit' without also setting a 'watermark_column' of Long type.")); + } + } + + 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/IncrementalScanSource.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java index 9ce210a126f4..e27028d6d1f7 100644 --- 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 @@ -22,6 +22,8 @@ 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; @@ -68,16 +70,24 @@ public PCollection expand(PBegin input) { TableCache.getRefreshed( scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); - PCollection rows = - MoreObjects.firstNonNull(scanConfig.getStreaming(), false) - ? readUnbounded(input) - : readBounded(input, table); - return rows.setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); + return MoreObjects.firstNonNull(scanConfig.getStreaming(), false) + ? readUnbounded(input).setRowSchema(ReadUtils.outputCdcSchema(table.schema())) + : readBounded(input, table) + .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); } /** - * Watches for new snapshots and creates tasks for each range. Using GiB for autosharding, this - * groups tasks in batches of up to 4GB, then reads from each batch using an SDF. + * Watches for new snapshots and creates tasks for each range. Uses GiB (with auto-sharding) to + * groups tasks in batches of size {@link ReadUtils#MAX_FILE_BUFFER_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 @@ -85,9 +95,8 @@ private PCollection readUnbounded(PBegin input) { MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); return input .apply("Watch for Snapshots", new WatchForSnapshots(scanConfig, pollInterval)) - .setCoder(SnapshotInfo.getCoder()) - .apply( - "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) + .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( Window.>into(new GlobalWindows()) @@ -113,9 +122,7 @@ public KV> apply( return KV.of(input.getKey().getKey(), Lists.newArrayList(input.getValue())); } })) - .apply( - "Read Rows From Grouped Tasks", - ParDo.of(new ReadFromGroupedTasks(scanConfig.getCatalogConfig()))); + .apply("Read Rows From Grouped Tasks", ParDo.of(new ReadFromGroupedTasks(scanConfig))); } /** @@ -135,13 +142,15 @@ private PCollection readBounded(PBegin input, Table table) { ReadUtils.getToSnapshot(table, scanConfig), table.currentSnapshot().snapshotId()); return input .apply( - "Create Snapshot Ranges", - Create.of(ReadUtils.snapshotsBetween(table, scanConfig.getTableIdentifier(), from, to))) - .setCoder(SnapshotInfo.getCoder()) - .apply( - "Create Read Tasks", ParDo.of(new CreateReadTasksDoFn(scanConfig.getCatalogConfig()))) + "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.getCatalogConfig()))); + .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 index a628edd9ecb2..fff6e5b171d5 100644 --- 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 @@ -32,6 +32,8 @@ import org.apache.iceberg.Table; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Duration; import org.joda.time.Instant; /** @@ -40,17 +42,16 @@ *

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 split granularity is set to the incoming batch size, i.e. the number of potential splits - * equals the batch size. + *

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 IcebergCatalogConfig catalogConfig; + private final IcebergScanConfig scanConfig; private final Counter scanTasksCompleted = Metrics.counter(ReadFromGroupedTasks.class, "scanTasksCompleted"); - ReadFromGroupedTasks(IcebergCatalogConfig catalogConfig) { - this.catalogConfig = catalogConfig; + ReadFromGroupedTasks(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; } @ProcessElement @@ -60,10 +61,10 @@ public void process( OutputReceiver out) throws IOException, ExecutionException { String tableIdentifier = element.getKey().getTableIdentifierString(); - Instant timestamp = Instant.ofEpochMilli(element.getKey().getSnapshotTimestampMillis()); List readTasks = element.getValue(); - Table table = TableCache.get(tableIdentifier, catalogConfig.catalog()); + Table table = TableCache.get(tableIdentifier, scanConfig.getCatalogConfig().catalog()); Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); + Schema outputSchema = ReadUtils.outputCdcSchema(beamSchema); // SDF can split by the number of read tasks for (long taskIndex = tracker.currentRestriction().getFrom(); @@ -72,11 +73,20 @@ public void process( if (!tracker.tryClaim(taskIndex)) { return; } - FileScanTask task = readTasks.get((int) taskIndex).getFileScanTask(); + + ReadTask readTask = readTasks.get((int) taskIndex); + @Nullable String operation = readTask.getOperation(); + FileScanTask task = readTask.getFileScanTask(); + Instant outputTimestamp = ReadUtils.getReadTaskTimestamp(readTask, scanConfig); + try (CloseableIterable reader = ReadUtils.createReader(task, table)) { for (Record record : reader) { - Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); - out.outputWithTimestamp(row, timestamp); + Row row = + Row.withSchema(outputSchema) + .addValue(IcebergUtils.icebergRecordToBeamRow(beamSchema, record)) + .addValue(operation) + .build(); + out.outputWithTimestamp(row, outputTimestamp); } } scanTasksCompleted.inc(); @@ -99,4 +109,11 @@ public double getSize( } return size; } + + // infinite skew in case we encounter some files that don't support watermark column statistics, + // in which case we output a -inf timestamp. + @Override + public Duration getAllowedTimestampSkew() { + return Duration.millis(Long.MAX_VALUE); + } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index 2de86bfd4771..43681fa00821 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -29,6 +29,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; +import org.joda.time.Duration; import org.joda.time.Instant; /** @@ -38,31 +39,38 @@ */ // @BoundedPerElement class ReadFromTasks extends DoFn, Row> { - private final IcebergCatalogConfig catalogConfig; + private final IcebergScanConfig scanConfig; private final Counter scanTasksCompleted = Metrics.counter(ReadFromTasks.class, "scanTasksCompleted"); - ReadFromTasks(IcebergCatalogConfig catalogConfig) { - this.catalogConfig = catalogConfig; + ReadFromTasks(IcebergScanConfig scanConfig) { + this.scanConfig = scanConfig; } @ProcessElement public void process(@Element KV element, OutputReceiver out) throws IOException, ExecutionException { String tableIdentifier = element.getKey().getTableIdentifierString(); - Instant timestamp = Instant.ofEpochMilli(element.getKey().getSnapshotTimestampMillis()); ReadTask readTask = element.getValue(); - Table table = TableCache.get(tableIdentifier, catalogConfig.catalog()); + Table table = TableCache.get(tableIdentifier, scanConfig.getCatalogConfig().catalog()); Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); + Instant outputTimestamp = ReadUtils.getReadTaskTimestamp(readTask, scanConfig); FileScanTask task = readTask.getFileScanTask(); try (CloseableIterable reader = ReadUtils.createReader(task, table)) { for (Record record : reader) { Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); - out.outputWithTimestamp(row, timestamp); + out.outputWithTimestamp(row, outputTimestamp); } } scanTasksCompleted.inc(); } + + // infinite skew in case we encounter some files that don't support watermark column statistics, + // in which case we output a -inf timestamp. + @Override + public Duration getAllowedTimestampSkew() { + return Duration.millis(Long.MAX_VALUE); + } } 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 index 940fcd9adb65..4a155eae64c3 100644 --- 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 @@ -27,6 +27,7 @@ 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 @@ -54,6 +55,10 @@ static Builder builder() { abstract long getByteSize(); + abstract @Nullable String getOperation(); + + abstract long getSnapshotTimestampMillis(); + @SchemaIgnore FileScanTask getFileScanTask() { if (cachedFileScanTask == null) { @@ -68,6 +73,10 @@ abstract static class Builder { 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 index ce30e96bf58c..2e5c37fdcc19 100644 --- 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 @@ -48,14 +48,10 @@ static Builder builder() { abstract String getTableIdentifierString(); - abstract long getSnapshotTimestampMillis(); - @AutoValue.Builder abstract static class Builder { abstract Builder setTableIdentifierString(String table); - abstract Builder setSnapshotTimestampMillis(long millis); - 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 index 037d6be0265f..72bcf21405b1 100644 --- 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 @@ -17,17 +17,31 @@ */ 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 com.google.common.annotations.VisibleForTesting; +import java.nio.ByteBuffer; 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.concurrent.TimeUnit; import java.util.function.BiFunction; import java.util.stream.Collectors; +import org.apache.beam.sdk.annotations.Internal; 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.transforms.windowing.BoundedWindow; +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; @@ -46,23 +60,52 @@ import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.parquet.ParquetReader; +import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; 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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +/** Internal helper class for source operations. */ +@Internal +@VisibleForTesting public class ReadUtils { + private static final Logger LOG = LoggerFactory.getLogger(ReadUtils.class); + // default is 8MB. keep this low to avoid overwhelming memory - private static final int MAX_FILE_BUFFER_SIZE = 1 << 20; // 1MB + 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"; + static final String DEFAULT_WATERMARK_TIME_UNIT = TimeUnit.MICROSECONDS.name(); + + /** 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)); + } public static ParquetReader createReader(FileScanTask task, Table table) { String filePath = task.file().path().toString(); @@ -178,8 +221,80 @@ static List snapshotsBetween( snapshotId -> snapshotId != from ? table.snapshot(snapshotId) : null))) .stream() .map(s -> SnapshotInfo.fromSnapshot(s, tableIdentifier)) + .sorted(Comparator.comparingLong(SnapshotInfo::getSequenceNumber)) .collect(Collectors.toList()); - Collections.reverse(snapshotIds); + return snapshotIds; } + + static @Nullable Long getLowerBoundTimestampMillis( + FileScanTask fileScanTask, String watermarkColumnName, @Nullable String watermarkTimeUnit) { + Types.NestedField watermarkColumn = fileScanTask.schema().findField(watermarkColumnName); + int watermarkColumnId = watermarkColumn.fieldId(); + @Nullable Map lowerBounds = fileScanTask.file().lowerBounds(); + + if (lowerBounds != null && lowerBounds.containsKey(watermarkColumnId)) { + TimeUnit timeUnit = + TimeUnit.valueOf( + MoreObjects.firstNonNull(watermarkTimeUnit, DEFAULT_WATERMARK_TIME_UNIT) + .toUpperCase()); + return timeUnit.toMillis( + Conversions.fromByteBuffer( + Types.LongType.get(), checkStateNotNull(lowerBounds.get(watermarkColumnId)))); + } else { + LOG.warn( + "Could not find statistics for watermark column '{}' in file '{}'.", + watermarkColumnName, + fileScanTask.file().path()); + return null; + } + } + + /** + * Returns the output timestamp associated with this read task. + * + *

If a watermark column is not specified, we fall back on the snapshot's commit timestamp. + * + *

If a watermark column is specified, we attempt to fetch it from the file's stats. If that + * information isn't available for whatever reason, we default to -inf. + */ + static Instant getReadTaskTimestamp(ReadTask readTask, IcebergScanConfig scanConfig) { + long millis; + @Nullable String watermarkColumn = scanConfig.getWatermarkColumn(); + if (watermarkColumn != null) { + millis = + MoreObjects.firstNonNull( + ReadUtils.getLowerBoundTimestampMillis( + readTask.getFileScanTask(), watermarkColumn, scanConfig.getWatermarkTimeUnit()), + BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); + System.out.println("xxx using file millis " + millis); + } else { + millis = readTask.getSnapshotTimestampMillis(); + System.out.println("xxx using snapshot millis " + millis); + } + return Instant.ofEpochMilli(millis); + } + + 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/WatchForSnapshots.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WatchForSnapshots.java index 17a337afd03c..5ca124b8d4ba 100644 --- 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 @@ -19,27 +19,25 @@ import static org.apache.beam.sdk.transforms.Watch.Growth.PollResult; -import java.util.Collections; +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.Gauge; import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Watch; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; 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.sdk.values.TypeDescriptor; 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; /** * Keeps watch over an Iceberg table and continuously outputs a range of snapshots, at the specified @@ -47,7 +45,7 @@ * *

A downstream transform will create a list of read tasks for each range. */ -class WatchForSnapshots extends PTransform> { +class WatchForSnapshots extends PTransform>>> { private final Duration pollInterval; private final IcebergScanConfig scanConfig; @@ -57,19 +55,17 @@ class WatchForSnapshots extends PTransform> { } @Override - public PCollection expand(PBegin input) { + public PCollection>> expand(PBegin input) { return input .apply(Create.of(scanConfig.getTableIdentifier())) .apply( "Watch for Snapshots", Watch.growthOf(new SnapshotPollFn(scanConfig)) .withPollInterval(pollInterval) - .withOutputCoder(SnapshotInfo.getCoder())) - .apply( - "Strip key", MapElements.into(TypeDescriptor.of(SnapshotInfo.class)).via(KV::getValue)); + .withOutputCoder(ListCoder.of(SnapshotInfo.getCoder()))); } - private static class SnapshotPollFn extends Watch.Growth.PollFn { + private static class SnapshotPollFn extends Watch.Growth.PollFn> { private final Gauge latestSnapshot = Metrics.gauge(SnapshotPollFn.class, "latestSnapshot"); private final IcebergScanConfig scanConfig; private @Nullable Long fromSnapshotId; @@ -79,7 +75,7 @@ private static class SnapshotPollFn extends Watch.Growth.PollFn apply(String tableIdentifier, Context c) { + public PollResult> apply(String tableIdentifier, Context c) { // fetch a fresh table to catch updated snapshots Table table = TableCache.getRefreshed(tableIdentifier, scanConfig.getCatalogConfig().catalog()); @@ -106,21 +102,16 @@ public PollResult apply(String tableIdentifier, Context c) { } /** Returns an appropriate PollResult based on the requested boundedness. */ - private PollResult getPollResult( + private PollResult> getPollResult( @Nullable List snapshots, boolean isComplete) { - List> timestampedValues = - snapshots == null - ? Collections.emptyList() - : snapshots.stream() - .map( - snapshot -> - TimestampedValue.of( - snapshot, Instant.ofEpochMilli(snapshot.getTimestampMillis()))) - .collect(Collectors.toList()); + List>> timestampedSnapshots = new ArrayList<>(1); + if (snapshots != null) { + timestampedSnapshots.add(TimestampedValue.of(snapshots, BoundedWindow.TIMESTAMP_MIN_VALUE)); + } return isComplete - ? PollResult.complete(timestampedValues) // stop at specified snapshot - : PollResult.incomplete(timestampedValues); // continue forever + ? 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..7cf3ed875941 --- /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.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 IcebergReadSchemaTransformProvider().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) + .withFieldValue("watermark_column", "abc") + .withFieldValue("watermark_time_unit", "nanoseconds") + .build(); + + new IcebergReadSchemaTransformProvider().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(UNBOUNDED)); + 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 testReadUsingManagedTransform() 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", + 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 986197d0ec20..195634675973 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,7 +17,10 @@ */ 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.ReadUtils.RECORD; 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; @@ -29,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.generic.GenericData; @@ -50,6 +54,7 @@ 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; @@ -71,6 +76,7 @@ import org.apache.parquet.hadoop.ParquetWriter; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; +import org.joda.time.Instant; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -108,8 +114,10 @@ public static Iterable data() { static class PrintRow extends DoFn { @ProcessElement - public void process(@Element Row row, OutputReceiver output) throws Exception { + public void process(@Element Row row, @Timestamp Instant timestamp, OutputReceiver output) + throws Exception { LOG.info("Got row {}", row); + LOG.info("timestamp: " + timestamp); output.output(row); } } @@ -124,7 +132,7 @@ public void testFailWhenBothStartingSnapshotAndTimestampAreSet() { thrown.expect(IllegalArgumentException.class); thrown.expectMessage( - "Invalid source configuration: Only one of 'from_timestamp' or 'from_snapshot' can be set"); + "Invalid source configuration: only one of 'from_timestamp' or 'from_snapshot' can be set"); read.expand(PBegin.in(testPipeline)); } @@ -138,7 +146,7 @@ public void testFailWhenBothEndingSnapshotAndTimestampAreSet() { thrown.expect(IllegalArgumentException.class); thrown.expectMessage( - "Invalid source configuration: Only one of 'to_timestamp' or 'to_snapshot' can be set"); + "Invalid source configuration: only one of 'to_timestamp' or 'to_snapshot' can be set"); read.expand(PBegin.in(testPipeline)); } @@ -175,6 +183,68 @@ public void testFailWhenPollIntervalIsSetOnBatchRead() { read.expand(PBegin.in(testPipeline)); } + @Test + public void testFailWhenWatermarkColumnDoesNotExist() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + warehouse.createTable(tableId, TestFixtures.SCHEMA); + IcebergIO.ReadRows read = + IcebergIO.readRows(catalogConfig()).from(tableId).withWatermarkColumn("unknown"); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid source configuration: the specified 'watermark_column' field does not exist: 'unknown'"); + read.expand(PBegin.in(testPipeline)); + } + + @Test + public void testFailWithInvalidWatermarkColumnType() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + warehouse.createTable(tableId, TestFixtures.SCHEMA); + IcebergIO.ReadRows read = + IcebergIO.readRows(catalogConfig()).from(tableId).withWatermarkColumn("data"); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid source configuration: invalid 'watermark_column' type: string. " + + "Valid types are [long, timestamp, timestamptz]"); + read.expand(PBegin.in(testPipeline)); + } + + @Test + public void testFailWhenWatermarkColumnMissingMetrics() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); + table.updateProperties().set("write.metadata.metrics.default", "none").commit(); + IcebergIO.ReadRows read = + IcebergIO.readRows(catalogConfig()).from(tableId).withWatermarkColumn("id"); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage("Invalid source configuration: source table"); + thrown.expectMessage( + "not configured to capture lower bound metrics for the specified watermark column 'id'."); + thrown.expectMessage("found 'none'"); + + read.expand(PBegin.in(testPipeline)); + } + + @Test + public void testFailWhenWatermarkTimeUnitUsedWithoutSpecifyingColumn() { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + warehouse.createTable(tableId, TestFixtures.SCHEMA); + IcebergIO.ReadRows read = + IcebergIO.readRows(catalogConfig()).from(tableId).withWatermarkTimeUnit("hours"); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Invalid source configuration: cannot set 'watermark_time_unit' without " + + "also setting a 'watermark_column' of Long type"); + read.expand(PBegin.in(testPipeline)); + } + @Test public void testSimpleScan() throws Exception { TableIdentifier tableId = @@ -421,6 +491,88 @@ public void testBatchReadBetweenTimestamps() throws IOException { runReadWithBoundary(false, false); } + @Test + public void testWatermarkColumn() throws IOException { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); + + // configure the table to capture full metrics + simpleTable + .updateProperties() + .set(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "id", "full") + .commit(); + + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "file1s4.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT4)) + .commit(); + simpleTable + .newFastAppend() + .appendFile( + warehouse.writeRecords( + "file2s4.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT4)) + .appendFile( + warehouse.writeRecords( + "file3s4.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT4)) + .commit(); + + IcebergIO.ReadRows readRows = + IcebergIO.readRows(catalogConfig()) + .from(tableId) + .streaming(true) + .withStartingStrategy(StartingStrategy.EARLIEST) + .toSnapshot(simpleTable.currentSnapshot().snapshotId()) + .withWatermarkColumn("id") + .withWatermarkTimeUnit("days"); + PCollection output = testPipeline.apply(readRows); + output.apply( + ParDo.of( + new CheckWatermarks( + TestFixtures.FILE1SNAPSHOT4_DATA, + TestFixtures.FILE2SNAPSHOT4_DATA, + TestFixtures.FILE3SNAPSHOT4_DATA))); + testPipeline.run().waitUntilFinish(); + } + + static class CheckWatermarks extends DoFn { + long file1Watermark; + long file2Watermark; + long file3Watermark; + + CheckWatermarks( + List> data1, + List> data2, + List> data3) { + file1Watermark = lowestMillisOf(data1); + file2Watermark = lowestMillisOf(data2); + file3Watermark = lowestMillisOf(data3); + } + + @ProcessElement + public void process(@Element Row row, @Timestamp Instant timestamp) { + Row record = checkStateNotNull(row.getRow(RECORD)); + long id = checkStateNotNull(record.getInt64("id")); + long expectedMillis = TimeUnit.DAYS.toMillis(id); + long actualMillis = timestamp.getMillis(); + + if (expectedMillis >= file3Watermark) { + assertEquals(file3Watermark, actualMillis); + } else if (expectedMillis >= file2Watermark) { + assertEquals(file2Watermark, actualMillis); + } else { + assertEquals(file1Watermark, actualMillis); + } + } + + private static long lowestMillisOf(List> data) { + long lowestId = data.stream().mapToLong(m -> (long) m.get("id")).min().getAsLong(); + return TimeUnit.DAYS.toMillis(lowestId); + } + } + public void runWithStartingStrategy(@Nullable StartingStrategy strategy, boolean streaming) throws IOException { assumeTrue(useIncrementalScan); @@ -448,7 +600,17 @@ public void runWithStartingStrategy(@Nullable StartingStrategy strategy, boolean } PCollection output = testPipeline.apply(readRows); - + if (streaming) { + PAssert.that(output) + .satisfies( + rows -> { + for (Row row : rows) { + assertEquals(DataOperations.APPEND, checkStateNotNull(row.getString(OPERATION))); + } + return null; + }); + output = output.apply(ReadUtils.extractRecords()); + } PCollection.IsBounded expectedBoundedness = streaming ? PCollection.IsBounded.UNBOUNDED : PCollection.IsBounded.BOUNDED; assertEquals(expectedBoundedness, output.isBounded()); @@ -490,9 +652,19 @@ public void runReadWithBoundary(boolean useSnapshotBoundary, boolean streaming) .toTimestamp(thirdSnapshot.timestampMillis() + 1); } - PCollection output = - testPipeline.apply(readRows).apply(ParDo.of(new PrintRow())).setRowSchema(schema); - + PCollection output = testPipeline.apply(readRows); + output = output.apply(ParDo.of(new PrintRow())).setRowSchema(output.getSchema()); + if (streaming) { + PAssert.that(output) + .satisfies( + rows -> { + for (Row row : rows) { + assertEquals(DataOperations.APPEND, checkStateNotNull(row.getString(OPERATION))); + } + return null; + }); + output = output.apply(ReadUtils.extractRecords()); + } PCollection.IsBounded expectedBoundedness = streaming ? PCollection.IsBounded.UNBOUNDED : PCollection.IsBounded.BOUNDED; assertEquals(expectedBoundedness, output.isBounded()); 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 8a461783e72b..4b483b57c635 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 @@ -22,7 +22,6 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; -import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -35,9 +34,7 @@ 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.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.Record; @@ -45,13 +42,9 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameter; -import org.junit.runners.Parameterized.Parameters; import org.yaml.snakeyaml.Yaml; -@RunWith(Parameterized.class) +/** Tests for {@link IcebergReadSchemaTransformProvider}. */ public class IcebergReadSchemaTransformProviderTest { @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -60,39 +53,27 @@ public class IcebergReadSchemaTransformProviderTest { @Rule public TestPipeline testPipeline = TestPipeline.create(); - @Parameters - public static Iterable data() { - return Arrays.asList(new Object[][] {{false}, {true}}); - } - - @Parameter(0) - public boolean useIncrementalScan; - @Test public void testBuildTransformWithRow() { Map properties = new HashMap<>(); properties.put("type", CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP); properties.put("warehouse", "test_location"); - Row.FieldValueBuilder configBuilder = + Row config = Row.withSchema(new IcebergReadSchemaTransformProvider().configurationSchema()) .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_name", "test-name") - .withFieldValue("catalog_properties", properties); - - if (useIncrementalScan) { - configBuilder = - configBuilder - .withFieldValue("from_snapshot", 123L) - .withFieldValue("to_snapshot", 456L) - .withFieldValue("from_timestamp", 123L) - .withFieldValue("to_timestamp", 456L) - .withFieldValue("starting_strategy", "earliest") - .withFieldValue("streaming", true) - .withFieldValue("poll_interval_seconds", 789); - } - - new IcebergReadSchemaTransformProvider().from(configBuilder.build()); + .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("watermark_column", "abc") + .withFieldValue("watermark_time_unit", "nanoseconds") + .build(); + + new IcebergReadSchemaTransformProvider().from(config); } @Test @@ -115,10 +96,6 @@ public void testSimpleScan() throws Exception { .setCatalogName("name") .setCatalogProperties(properties); - if (useIncrementalScan) { - readConfigBuilder = - readConfigBuilder.setToSnapshot(simpleTable.currentSnapshot().snapshotId()); - } final List expectedRows = expectedRecords.stream() .flatMap(List::stream) @@ -159,17 +136,6 @@ public void testReadUsingManagedTransform() throws Exception { + " warehouse: %s", identifier, CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, warehouse.location); - if (useIncrementalScan) { - // only read files that were added in the second and third snapshots, - // ignoring the first and fourth snapshots. - expectedRecords = expectedRecords.subList(3, 9); - - List snapshots = Lists.newArrayList(simpleTable.snapshots()); - long second = snapshots.get(1).snapshotId(); - long third = snapshots.get(2).snapshotId(); - - yamlConfig += String.format("\n" + "from_snapshot: %s\n" + "to_snapshot: %s", second, third); - } final List expectedRows = expectedRecords.stream() .flatMap(List::stream) 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..8cb403bec1f9 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,40 @@ 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("watermark_column", "id") + .withFieldValue("watermark_time_unit", "nanoseconds") + .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 +119,7 @@ public void testReCreateWriteTransformFromRow() { IcebergWriteSchemaTransform writeTransformFromRow = translator.fromConfigRow(row, PipelineOptionsFactory.create()); - assertEquals(transformConfigRow, writeTransformFromRow.getConfigurationRow()); + assertEquals(WRITE_CONFIG_ROW, writeTransformFromRow.getConfigurationRow()); } @Test @@ -108,16 +134,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 +164,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 +172,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 +187,7 @@ public void testReCreateReadTransformFromRow() { IcebergReadSchemaTransform readTransformFromRow = translator.fromConfigRow(row, PipelineOptionsFactory.create()); - assertEquals(transformConfigRow, readTransformFromRow.getConfigurationRow()); + assertEquals(READ_CONFIG_ROW, readTransformFromRow.getConfigurationRow()); } @Test @@ -193,11 +202,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 +247,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 index a2002b1e3a5d..107224926969 100644 --- 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 @@ -24,11 +24,14 @@ import java.util.List; import java.util.Map; import java.util.UUID; +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; @@ -83,4 +86,43 @@ public void testCreateReader() throws IOException { } assertEquals(data.size(), numFiles); } + + @Test + public void testSnapshotsBetween() throws IOException { + TableIdentifier tableId = + TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + 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())); + } } diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java index b9e0801df82b..ecab8efb23ae 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java @@ -20,6 +20,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; +import com.google.api.client.util.Lists; import com.google.api.services.bigquery.model.TableRow; import java.io.IOException; import java.util.Arrays; @@ -37,11 +38,15 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.util.SnapshotUtil; import org.junit.AfterClass; import org.junit.BeforeClass; +import org.junit.Ignore; import org.junit.Test; public class BigQueryMetastoreCatalogIT extends IcebergCatalogBaseIT { @@ -104,6 +109,79 @@ public Map managedIcebergConfig(String tableId) { .build(); } + // Schema schema = Schema.builder().addInt32Field("id").addStringField("name").build(); + // List rows = + // IntStream.range(0, 10) + // .mapToObj(i -> Row.withSchema(schema).addValues(i, "val_" + i).build()) + // .collect(Collectors.toList()); + + // @Override + // public Integer numRecords() { + // return 2; + // } + + @Test + @Ignore + public void testTemp() throws IOException, InterruptedException { + // Map config = managedIcebergConfig(tableId()); + // writePipeline.getOptions().as(DirectOptions.class).setTargetParallelism(1); + // PCollection input = writePipeline.apply(Create.of(rows)).setRowSchema(schema); + // input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); + // writePipeline.run().waitUntilFinish(); + + Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + populateTable(table); + populateTable(table); + populateTable(table); + populateTable(table); + populateTable(table); + populateTable(table); + populateTable(table); + populateTable(table); + List snapshots = + Lists.newArrayList(table.snapshots()).stream() + .map(Snapshot::snapshotId) + .collect(Collectors.toList()); + System.out.println("xxx snapshots: " + snapshots); + // [6011121439735012896, 1895704562814642328, 589404649896567915, 4769353889403890084, + // 5320805826111823286, 543163132885668788, 5985342948512626443, 5088987033302496149] + // between 1895704562814642328 and 5985342948512626443: + // [5985342948512626443, 543163132885668788, 5320805826111823286, 4769353889403890084, + // 589404649896567915, 1895704562814642328, 6011121439735012896] + List between = + Lists.newArrayList(SnapshotUtil.snapshotIdsBetween(table, -1, snapshots.get(6))); + System.out.printf("xxx between %s and %s:\n%s%n", snapshots.get(1), snapshots.get(6), between); + + // String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + // NameMapping mapping = + // nameMapping != null ? NameMappingParser.fromJson(nameMapping) : + // NameMapping.empty(); + + // try (CloseableIterable tasks = + // table.newIncrementalChangelogScan().planFiles()) { + // for (ChangelogScanTask task : tasks) { + // + // System.out.printf( + // "xxx change log:\n" + " snapshot: %s\n" + " change type: %s\n ordinal: + // %s\n", + // task.commitSnapshotId(), task.operation(), task.changeOrdinal()); + // } + // } + // + // TableScan scan = table.newScan().filter(Expressions.lessThan("id", 5)); + // int count = 0; + // try (CloseableIterable tasks = scan.planFiles()) { + // for (FileScanTask fileScanTask : tasks) { + // String path = fileScanTask.file().path().toString(); + // String name = Paths.get(path.replace("gs://", "")).getFileName().toString(); + // System.out.println("xxx file path: " + name); + // count++; + // } + // } + + // System.out.println("xxx num files: " + count); + } + @Test public void testWriteToPartitionedAndValidateWithBQQuery() throws IOException, InterruptedException { @@ -119,9 +197,9 @@ public void testWriteToPartitionedAndValidateWithBQQuery() // Write with Beam Map config = managedIcebergConfig(tableId()); - PCollection input = writePipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); + PCollection input = pipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - writePipeline.run().waitUntilFinish(); + pipeline.run().waitUntilFinish(); // Fetch records using a BigQuery query and validate BigqueryClient bqClient = new BigqueryClient(getClass().getSimpleName()); @@ -144,4 +222,21 @@ public void testWriteToPartitionedAndValidateWithBQQuery() .collect(Collectors.toList()); assertThat(beamRows, containsInAnyOrder(inputRows.stream().map(rowFilter::filter).toArray())); } + + // private static class PrintElements extends PTransform, PCollection> { + // @Override + // public PCollection expand(PCollection input) { + // return input.apply( + // MapElements.into(TypeDescriptors.voids()) + // .via( + // r -> { + // System.out.println("xxx " + r); + // return null; + // })); + // } + // + // static PrintElements rows() { + // return new PrintElements(); + // } + // } } 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 6931a3bb445d..a59c8c48374d 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 @@ -18,6 +18,8 @@ package org.apache.beam.sdk.io.iceberg.catalog; 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; @@ -35,15 +37,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; @@ -91,6 +96,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; @@ -150,6 +157,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", @@ -201,10 +209,9 @@ public void cleanUp() throws Exception { protected static final GcpOptions OPTIONS = TestPipeline.testingPipelineOptions().as(GcpOptions.class); private static final String RANDOM = UUID.randomUUID().toString(); - @Rule public TestPipeline writePipeline = TestPipeline.create(); - @Rule public TestPipeline readPipeline = TestPipeline.create(); + @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 = @@ -239,6 +246,9 @@ public void cleanUp() throws Exception { private static final SimpleFunction ROW_FUNC = new SimpleFunction() { + final long OFFSET_2025_MILLIS = + LocalDate.parse("2025-01-01").toDateTime(LocalTime.MIDNIGHT).getMillis(); + @Override public Row apply(Long num) { String strNum = Long.toString(num); @@ -254,6 +264,7 @@ public Row apply(Long num) { .addValue(Float.valueOf(strNum + "." + strNum)) .build(); + long timestampMillis = OFFSET_2025_MILLIS + TimeUnit.MICROSECONDS.toHours(num); return Row.withSchema(BEAM_SCHEMA) .addValue("value_" + strNum) .addValue(String.valueOf((char) (97 + num % 5))) @@ -264,8 +275,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(); @@ -285,7 +297,7 @@ 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); } @@ -294,6 +306,7 @@ private List populateTable(Table table, @Nullable String charOverride) thro 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; @@ -392,31 +405,31 @@ public void testRead() throws Exception { Map config = managedIcebergConfig(tableId()); PCollection rows = - writePipeline - .apply(Managed.read(Managed.ICEBERG).withConfig(config)) - .getSinglePCollection(); + pipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); PAssert.that(rows).containsInAnyOrder(expectedRows); - writePipeline.run().waitUntilFinish(); + pipeline.run().waitUntilFinish(); } @Test - public void testUnboundedRead() throws Exception { + 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()); + config.put("watermark_column", "datetime"); PCollection rows = - writePipeline - .apply(Managed.read(Managed.ICEBERG).withConfig(config)) - .getSinglePCollection(); + pipeline + .apply(Managed.read(Managed.ICEBERG_CDC).withConfig(config)) + .getSinglePCollection() + .apply(ReadUtils.extractRecords()); + assertThat(rows.isBounded(), equalTo(UNBOUNDED)); PAssert.that(rows).containsInAnyOrder(expectedRows); - writePipeline.run().waitUntilFinish(); + pipeline.run().waitUntilFinish(); } @Test @@ -430,43 +443,48 @@ public void testStreamingReadBetweenTimestamps() throws Exception { } @Test - public void testWriteRead() { + public void testWriteRead() throws IOException { + Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + List expectedRows = populateTable(table); Map config = managedIcebergConfig(tableId()); - PCollection input = writePipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); - input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - writePipeline.run().waitUntilFinish(); - PCollection output = - readPipeline.apply(Managed.read(Managed.ICEBERG).withConfig(config)).getSinglePCollection(); - PAssert.that(output).containsInAnyOrder(inputRows); - readPipeline.run().waitUntilFinish(); + pipeline + .apply("read", Managed.read(Managed.ICEBERG).withConfig(config)) + .getSinglePCollection() + .apply("write", Managed.write(Managed.ICEBERG).withConfig(config)); + pipeline.run().waitUntilFinish(); + + List returnedRecords = readRecords(table); + assertThat( + returnedRecords, + containsInAnyOrder(expectedRows.stream().map(RECORD_FUNC::apply).toArray())); } @Test - public void testWriteReadStreaming() { - int numRecords = numRecords(); + public void testReadWriteStreaming() throws IOException { + Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA); + List expectedRows = populateTable(table); + Map config = managedIcebergConfig(tableId()); - Map writeConfig = new HashMap<>(config); - writeConfig.put("triggering_frequency_seconds", 5); - PCollection input = - writePipeline - .apply(getStreamingSource()) - .apply( - MapElements.into(TypeDescriptors.rows()) - .via(instant -> ROW_FUNC.apply(instant.getMillis() % numRecords))) - .setRowSchema(BEAM_SCHEMA); - input.apply(Managed.write(Managed.ICEBERG).withConfig(writeConfig)); - writePipeline.run().waitUntilFinish(); Map readConfig = new HashMap<>(config); - readConfig.put("streaming", true); readConfig.put("to_timestamp", System.currentTimeMillis()); - PCollection output = - readPipeline - .apply(Managed.read(Managed.ICEBERG).withConfig(readConfig)) - .getSinglePCollection(); - PAssert.that(output).containsInAnyOrder(inputRows); - readPipeline.run().waitUntilFinish(); + readConfig.put("watermark_column", "datetime_tz"); + + Map writeConfig = new HashMap<>(config); + writeConfig.put("triggering_frequency_seconds", 5); + + pipeline + .apply("streaming read", Managed.read(Managed.ICEBERG_CDC).withConfig(readConfig)) + .getSinglePCollection() + .apply(ReadUtils.extractRecords()) + .apply("streaming write", Managed.write(Managed.ICEBERG).withConfig(writeConfig)); + pipeline.run().waitUntilFinish(); + + List returnedRecords = readRecords(table); + assertThat( + returnedRecords, + containsInAnyOrder(expectedRows.stream().map(RECORD_FUNC::apply).toArray())); } @Test @@ -474,9 +492,9 @@ public void testWrite() throws IOException { // Write with Beam // Expect the sink to create the table Map config = managedIcebergConfig(tableId()); - PCollection input = writePipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); + PCollection input = pipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - writePipeline.run().waitUntilFinish(); + pipeline.run().waitUntilFinish(); Table table = catalog.loadTable(TableIdentifier.parse(tableId())); assertTrue(table.schema().sameSchema(ICEBERG_SCHEMA)); @@ -502,9 +520,9 @@ public void testWriteToPartitionedTable() throws IOException { // Write with Beam Map config = managedIcebergConfig(tableId()); - PCollection input = writePipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); + PCollection input = pipeline.apply(Create.of(inputRows)).setRowSchema(BEAM_SCHEMA); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - writePipeline.run().waitUntilFinish(); + pipeline.run().waitUntilFinish(); // Read back and check records are correct List returnedRecords = readRecords(table); @@ -531,17 +549,17 @@ public void testStreamingWrite() throws IOException { // create elements from longs in range [0, 1000) PCollection input = - writePipeline + pipeline .apply(getStreamingSource()) .apply( MapElements.into(TypeDescriptors.rows()) .via(instant -> ROW_FUNC.apply(instant.getMillis() % numRecords))) .setRowSchema(BEAM_SCHEMA); - assertThat(input.isBounded(), equalTo(IsBounded.UNBOUNDED)); + assertThat(input.isBounded(), equalTo(UNBOUNDED)); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - writePipeline.run().waitUntilFinish(); + pipeline.run().waitUntilFinish(); List returnedRecords = readRecords(table); assertThat( @@ -561,7 +579,7 @@ public void testStreamingWriteWithPriorWindowing() throws IOException { // over a span of 10 seconds, create elements from longs in range [0, 1000) PCollection input = - writePipeline + pipeline .apply(getStreamingSource()) .apply( Window.into(FixedWindows.of(Duration.standardSeconds(1))) @@ -571,10 +589,10 @@ public void testStreamingWriteWithPriorWindowing() throws IOException { .via(instant -> ROW_FUNC.apply(instant.getMillis() % numRecords))) .setRowSchema(BEAM_SCHEMA); - assertThat(input.isBounded(), equalTo(IsBounded.UNBOUNDED)); + assertThat(input.isBounded(), equalTo(UNBOUNDED)); input.apply(Managed.write(Managed.ICEBERG).withConfig(config)); - writePipeline.run().waitUntilFinish(); + pipeline.run().waitUntilFinish(); List returnedRecords = readRecords(table); assertThat( @@ -644,17 +662,17 @@ private void writeToDynamicDestinations( if (streaming) { writeConfig.put("triggering_frequency_seconds", 5); input = - writePipeline + pipeline .apply(getStreamingSource()) .apply( MapElements.into(TypeDescriptors.rows()) .via(instant -> ROW_FUNC.apply(instant.getMillis() % numRecords))); } else { - input = writePipeline.apply(Create.of(inputRows)); + input = pipeline.apply(Create.of(inputRows)); } input.setRowSchema(BEAM_SCHEMA).apply(Managed.write(Managed.ICEBERG).withConfig(writeConfig)); - writePipeline.run().waitUntilFinish(); + pipeline.run().waitUntilFinish(); Table table0 = catalog.loadTable(tableIdentifier0); Table table1 = catalog.loadTable(tableIdentifier1); @@ -727,6 +745,8 @@ public void runReadBetween(boolean useSnapshotBoundary, boolean streaming) throw populateTable(table, "d"); // fourth snapshot Map config = new HashMap<>(managedIcebergConfig(tableId())); + config.put("watermark_column", "nullable_long"); + config.put("watermark_time_unit", "days"); if (useSnapshotBoundary) { config.put("from_snapshot", from.snapshotId()); config.put("to_snapshot", to.snapshotId()); @@ -735,19 +755,19 @@ public void runReadBetween(boolean useSnapshotBoundary, boolean streaming) throw config.put("to_timestamp", to.timestampMillis() + 1); } - if (streaming) { - config.put("streaming", true); - } + String source = streaming ? Managed.ICEBERG_CDC : Managed.ICEBERG; PCollection rows = - writePipeline - .apply(Managed.read(Managed.ICEBERG).withConfig(config)) - .getSinglePCollection(); + pipeline.apply(Managed.read(source).withConfig(config)).getSinglePCollection(); + + if (streaming) { + rows = rows.apply(ReadUtils.extractRecords()); + } - IsBounded expectedBoundedness = streaming ? IsBounded.UNBOUNDED : IsBounded.BOUNDED; + IsBounded expectedBoundedness = streaming ? UNBOUNDED : BOUNDED; assertEquals(expectedBoundedness, rows.isBounded()); PAssert.that(rows).containsInAnyOrder(expectedRows); - writePipeline.run().waitUntilFinish(); + 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 8e7e0862eff4..4d5a3567bff6 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 @@ -85,6 +85,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"; @@ -92,6 +93,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(); From fbad86e5db054e1d2f2d569d82f9f5f916d1973f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 3 Mar 2025 14:37:57 -0500 Subject: [PATCH 16/32] cleanup --- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 6 +- .../catalog/BigQueryMetastoreCatalogIT.java | 95 ------------------- 2 files changed, 4 insertions(+), 97 deletions(-) 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 1151618fa2f2..1cea05c2c716 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 @@ -439,8 +439,10 @@ * .apply(ReadUtils.extractRecords()); * } * - *

The streaming source continuously polls the table for new append-only snapshots, with a default interval - * of 60 seconds. This can be overridden using {@code poll_interval_seconds}. + *

The streaming source continuously polls the table for new append-only snapshots, with a + * default interval of 60 seconds. This can be overridden using {@code + * poll_interval_seconds}. + * *

Note: Full CDC is not supported yet. * *

{@code
diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java
index ecab8efb23ae..3a8b47cb5a06 100644
--- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java
+++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/catalog/BigQueryMetastoreCatalogIT.java
@@ -20,7 +20,6 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 
-import com.google.api.client.util.Lists;
 import com.google.api.services.bigquery.model.TableRow;
 import java.io.IOException;
 import java.util.Arrays;
@@ -38,15 +37,11 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.iceberg.CatalogUtil;
 import org.apache.iceberg.PartitionSpec;
-import org.apache.iceberg.Snapshot;
-import org.apache.iceberg.Table;
 import org.apache.iceberg.catalog.Catalog;
 import org.apache.iceberg.catalog.Namespace;
 import org.apache.iceberg.catalog.TableIdentifier;
-import org.apache.iceberg.util.SnapshotUtil;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 public class BigQueryMetastoreCatalogIT extends IcebergCatalogBaseIT {
@@ -109,79 +104,6 @@ public Map managedIcebergConfig(String tableId) {
         .build();
   }
 
-  //  Schema schema = Schema.builder().addInt32Field("id").addStringField("name").build();
-  //  List rows =
-  //      IntStream.range(0, 10)
-  //          .mapToObj(i -> Row.withSchema(schema).addValues(i, "val_" + i).build())
-  //          .collect(Collectors.toList());
-
-  //  @Override
-  //  public Integer numRecords() {
-  //    return 2;
-  //  }
-
-  @Test
-  @Ignore
-  public void testTemp() throws IOException, InterruptedException {
-    //    Map config = managedIcebergConfig(tableId());
-    //    writePipeline.getOptions().as(DirectOptions.class).setTargetParallelism(1);
-    //    PCollection input = writePipeline.apply(Create.of(rows)).setRowSchema(schema);
-    //    input.apply(Managed.write(Managed.ICEBERG).withConfig(config));
-    //    writePipeline.run().waitUntilFinish();
-
-    Table table = catalog.createTable(TableIdentifier.parse(tableId()), ICEBERG_SCHEMA);
-    populateTable(table);
-    populateTable(table);
-    populateTable(table);
-    populateTable(table);
-    populateTable(table);
-    populateTable(table);
-    populateTable(table);
-    populateTable(table);
-    List snapshots =
-        Lists.newArrayList(table.snapshots()).stream()
-            .map(Snapshot::snapshotId)
-            .collect(Collectors.toList());
-    System.out.println("xxx snapshots: " + snapshots);
-    // [6011121439735012896, 1895704562814642328, 589404649896567915, 4769353889403890084,
-    // 5320805826111823286, 543163132885668788, 5985342948512626443, 5088987033302496149]
-    // between 1895704562814642328 and 5985342948512626443:
-    // [5985342948512626443, 543163132885668788, 5320805826111823286, 4769353889403890084,
-    // 589404649896567915, 1895704562814642328, 6011121439735012896]
-    List between =
-        Lists.newArrayList(SnapshotUtil.snapshotIdsBetween(table, -1, snapshots.get(6)));
-    System.out.printf("xxx between %s and %s:\n%s%n", snapshots.get(1), snapshots.get(6), between);
-
-    //    String nameMapping = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING);
-    //    NameMapping mapping =
-    //            nameMapping != null ? NameMappingParser.fromJson(nameMapping) :
-    // NameMapping.empty();
-
-    //    try (CloseableIterable tasks =
-    //        table.newIncrementalChangelogScan().planFiles()) {
-    //      for (ChangelogScanTask task : tasks) {
-    //
-    //        System.out.printf(
-    //            "xxx change log:\n" + "    snapshot: %s\n" + "    change type: %s\n    ordinal:
-    // %s\n",
-    //            task.commitSnapshotId(), task.operation(), task.changeOrdinal());
-    //      }
-    //    }
-    //
-    //    TableScan scan = table.newScan().filter(Expressions.lessThan("id", 5));
-    //    int count = 0;
-    //    try (CloseableIterable tasks = scan.planFiles()) {
-    //      for (FileScanTask fileScanTask : tasks) {
-    //        String path = fileScanTask.file().path().toString();
-    //        String name = Paths.get(path.replace("gs://", "")).getFileName().toString();
-    //        System.out.println("xxx file path: " + name);
-    //        count++;
-    //      }
-    //    }
-
-    //    System.out.println("xxx num files: " + count);
-  }
-
   @Test
   public void testWriteToPartitionedAndValidateWithBQQuery()
       throws IOException, InterruptedException {
@@ -222,21 +144,4 @@ public void testWriteToPartitionedAndValidateWithBQQuery()
             .collect(Collectors.toList());
     assertThat(beamRows, containsInAnyOrder(inputRows.stream().map(rowFilter::filter).toArray()));
   }
-
-  //  private static class PrintElements extends PTransform, PCollection> {
-  //    @Override
-  //    public PCollection expand(PCollection input) {
-  //      return input.apply(
-  //          MapElements.into(TypeDescriptors.voids())
-  //              .via(
-  //                  r -> {
-  //                    System.out.println("xxx " + r);
-  //                    return null;
-  //                  }));
-  //    }
-  //
-  //    static PrintElements rows() {
-  //      return new PrintElements();
-  //    }
-  //  }
 }

From 50f94975579889d8fbb0591f38f6cbc39070398c Mon Sep 17 00:00:00 2001
From: Ahmed Abualsaud 
Date: Mon, 3 Mar 2025 15:02:47 -0500
Subject: [PATCH 17/32] add guava import

---
 sdks/java/io/iceberg/build.gradle | 1 +
 1 file changed, 1 insertion(+)

diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle
index 32c95ea8260c..e904f4921721 100644
--- a/sdks/java/io/iceberg/build.gradle
+++ b/sdks/java/io/iceberg/build.gradle
@@ -44,6 +44,7 @@ def hive_version = "3.1.3"
 
 dependencies {
     implementation library.java.vendored_guava_32_1_2_jre
+    implementation library.java.guava
     implementation project(path: ":sdks:java:core", configuration: "shadow")
     implementation project(path: ":model:pipeline", configuration: "shadow")
     implementation library.java.avro

From 4f1f40b65c805fa19962fd462e108b07c45a1110 Mon Sep 17 00:00:00 2001
From: Ahmed Abualsaud 
Date: Mon, 3 Mar 2025 15:03:47 -0500
Subject: [PATCH 18/32] remove iceberg_cdc_read from xlang auto-wrapper gen

---
 sdks/standard_expansion_services.yaml | 1 +
 1 file changed, 1 insertion(+)

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'

From 633365cf45779d0ff1d4155b8a0409e76d128a0c Mon Sep 17 00:00:00 2001
From: Ahmed Abualsaud 
Date: Mon, 3 Mar 2025 15:09:52 -0500
Subject: [PATCH 19/32] fix javadoc

---
 .../org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

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
index e27028d6d1f7..6c41f102a660 100644
--- 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
@@ -78,7 +78,7 @@ public PCollection expand(PBegin input) {
 
   /**
    * Watches for new snapshots and creates tasks for each range. Uses GiB (with auto-sharding) to
-   * groups tasks in batches of size {@link ReadUtils#MAX_FILE_BUFFER_SIZE}, then reads from each
+   * groups tasks in batches of size {@link #MAX_FILES_BATCH_BYTE_SIZE}, then reads from each
    * batch using an SDF.
    *
    * 

Output schema is: From 37485f137368a93c0f5939a38f51572f1552693b Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 3 Mar 2025 15:22:28 -0500 Subject: [PATCH 20/32] cleanup --- .../java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java | 2 +- .../main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java | 1 - .../java/org/apache/beam/sdk/io/iceberg/ReadTaskDescriptor.java | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java index 794deb933275..26a0d87cbca5 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 @@ -295,7 +295,7 @@ void validate(Table table) { type); Set validTimeUnits = - Arrays.stream(TimeUnit.values()).map(Enum::name).collect(Collectors.toSet()); + Arrays.stream(TimeUnit.values()).map(TimeUnit::name).collect(Collectors.toSet()); checkArgument( validTimeUnits.contains(watermarkTimeUnit.toUpperCase()), error("invalid 'watermark_time_unit': %s. Please choose one of: %s"), diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index 43681fa00821..63d37b750b1c 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -37,7 +37,6 @@ * *

For each {@link ReadTask}, reads Iceberg {@link Record}s, and converts to Beam {@link Row}s. */ -// @BoundedPerElement class ReadFromTasks extends DoFn, Row> { private final IcebergScanConfig scanConfig; private final Counter scanTasksCompleted = 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 index 2e5c37fdcc19..6adbab83bf00 100644 --- 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 @@ -25,7 +25,7 @@ import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -/** Describes the table and snapshot a {@link ReadTask} belongs to. */ +/** Describes the table a {@link ReadTask} belongs to.*/ @DefaultSchema(AutoValueSchema.class) @AutoValue abstract class ReadTaskDescriptor { From 4ede0e8a4239c758e8d5ef37baad2d9d5ca4fee0 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 4 Mar 2025 08:10:58 -0500 Subject: [PATCH 21/32] spotless --- .../org/apache/beam/sdk/io/iceberg/IncrementalScanSource.java | 4 ++-- .../org/apache/beam/sdk/io/iceberg/ReadTaskDescriptor.java | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) 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 index 6c41f102a660..33d4eec75c3f 100644 --- 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 @@ -78,8 +78,8 @@ public PCollection expand(PBegin input) { /** * 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. + * groups tasks in batches of size {@link #MAX_FILES_BATCH_BYTE_SIZE}, then reads from each batch + * using an SDF. * *

Output schema is: * 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 index 6adbab83bf00..b7a9be32aba2 100644 --- 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 @@ -25,7 +25,7 @@ import org.apache.beam.sdk.schemas.annotations.DefaultSchema; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; -/** Describes the table a {@link ReadTask} belongs to.*/ +/** Describes the table a {@link ReadTask} belongs to. */ @DefaultSchema(AutoValueSchema.class) @AutoValue abstract class ReadTaskDescriptor { From db9fd63cf0d7d8cdc3716c9ba7795bfb861346a4 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 4 Mar 2025 14:50:27 -0500 Subject: [PATCH 22/32] use CDC schema for batch and streaming; re-introduce boolean 'streaming' option; doc updates --- ...IcebergCdcReadSchemaTransformProvider.java | 21 +- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 84 ++++-- .../IcebergReadSchemaTransformProvider.java | 90 +------ .../sdk/io/iceberg/IcebergScanConfig.java | 57 +++- .../sdk/io/iceberg/IncrementalScanSource.java | 23 +- .../sdk/io/iceberg/ReadFromGroupedTasks.java | 8 +- .../beam/sdk/io/iceberg/ReadFromTasks.java | 11 +- .../apache/beam/sdk/io/iceberg/ReadUtils.java | 2 +- ...ergCdcReadSchemaTransformProviderTest.java | 12 +- .../sdk/io/iceberg/IcebergIOReadTest.java | 245 +++++++++++++----- ...cebergReadSchemaTransformProviderTest.java | 7 - ...IcebergSchemaTransformTranslationTest.java | 1 + .../sdk/io/iceberg/TestDataWarehouse.java | 8 + .../iceberg/catalog/IcebergCatalogBaseIT.java | 38 +-- 14 files changed, 365 insertions(+), 242 deletions(-) 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 index c5f412cfcdd9..cf16d874038c 100644 --- 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 @@ -108,6 +108,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { IcebergIO.ReadRows readRows = IcebergIO.readRows(configuration.getIcebergCatalog()) + .withCdc() .from(TableIdentifier.parse(configuration.getTable())) .fromSnapshot(configuration.getFromSnapshot()) .toSnapshot(configuration.getToSnapshot()) @@ -116,7 +117,7 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .withStartingStrategy(strategy) .withWatermarkColumn(configuration.getWatermarkColumn()) .withWatermarkTimeUnit(configuration.getWatermarkTimeUnit()) - .streaming(true); + .streaming(configuration.getStreaming()); @Nullable Integer pollIntervalSeconds = configuration.getPollIntervalSeconds(); if (pollIntervalSeconds != null) { @@ -170,10 +171,6 @@ static Builder builder() { + "by setting a starting snapshot or timestamp. Defaults to earliest for batch, and latest for streaming.") abstract @Nullable String getStartingStrategy(); - @SchemaFieldDescription( - "The interval at which to poll for new snapshots. Defaults to 60 seconds.") - abstract @Nullable Integer getPollIntervalSeconds(); - @SchemaFieldDescription( "The column used to derive event time for tracking progress. Uses the snapshot's commit timestamp by default.") abstract @Nullable String getWatermarkColumn(); @@ -183,6 +180,14 @@ static Builder builder() { + "Check https://docs.oracle.com/javase/8/docs/api///?java/util/concurrent/TimeUnit.html for possible values.") abstract @Nullable String getWatermarkTimeUnit(); + @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); @@ -203,12 +208,14 @@ abstract static class Builder { abstract Builder setStartingStrategy(String strategy); - abstract Builder setPollIntervalSeconds(Integer pollInterval); - abstract Builder setWatermarkColumn(String column); abstract Builder setWatermarkTimeUnit(String watermarkTimeUnit); + abstract Builder setPollIntervalSeconds(Integer pollInterval); + + abstract Builder setStreaming(Boolean streaming); + abstract Configuration 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 1cea05c2c716..010cb5d48545 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 @@ -135,6 +135,8 @@ * *

Source-only Options

* + *

CDC Source options

+ * * * * @@ -198,13 +200,12 @@ *

Check {@link TimeUnit} for possible values. * *

- *
Parameter Type Description
- * - *

CDC Streaming Source options

- * - * * - * + * + * + * * * * @@ -219,7 +220,7 @@ * *

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: * *

Parameter Type Description {@code streaming} {@code boolean} + * Enables streaming reads. The source will continuously poll for snapshots forever. + *
{@code poll_interval_seconds}
* @@ -425,27 +426,30 @@ * Example of a simple batch read: * *
{@code
- * PCollection = pipeline
+ * PCollection rows = pipeline
  *     .apply(Managed.read(ICEBERG).withConfig(config))
  *     .getSinglePCollection();
  * }
* - * Example of a simple CDC streaming read: + * Example of a simple CDC read: * *
{@code
- * PCollection = pipeline
+ * PCollection output = pipeline
  *     .apply(Managed.read(ICEBERG_CDC).withConfig(config))
- *     .getSinglePCollection()
+ *     .getSinglePCollection();
+ *
+ * PCollection rows = output
  *     .apply(ReadUtils.extractRecords());
  * }
* - *

The streaming source continuously polls the table for new append-only snapshots, with a - * default interval of 60 seconds. This can be overridden using {@code - * poll_interval_seconds}. + *

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

Note: 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);
  * }
* @@ -457,13 +461,27 @@ *

Reading with {@code Managed.read(ICEBERG_CDC)} produces a {@code * PCollection} with the following schema: * - *

    - *
  • {@code "record"}: a Row representing the data record - *
  • {@code "operation"}: the snapshot operation associated with this record (e.g. - * "append", "replace", "delete") - *
+ *
+ * + * + * + * + * + * + * + * + * + * + * + * + * + *
Field Type Description
{@code record} {@code Beam Row} + * The data record. + *
{@code operation} {@code string} + * The snapshot operation associated with this record (e.g. "append", "replace", "delete") + *
* - *

Choosing a Starting Point

+ *

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 @@ -488,11 +506,11 @@ * "from_snapshot", 123456789L); * * PCollection = pipeline - * .apply(Managed.read(ICEBERG).withConfig(config)) + * .apply(Managed.read(ICEBERG_CDC).withConfig(config)) * .getSinglePCollection(); * }
* - *

Choosing an End Point

+ *

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 also be overridden with a @@ -515,13 +533,13 @@ * "to_timestamp", 987654321L); * * PCollection = pipeline - * .apply(Managed.read(ICEBERG).withConfig(config)) + * .apply(Managed.read(ICEBERG_CDC).withConfig(config)) * .getSinglePCollection(); * } * * Note: An end point can also be set when performing a streaming read. * - *

Handling Watermarks

+ *

Handling Watermarks (CDC only)

* * By default, a snapshot's commit timestamp is assigned to all the records it contains. * @@ -622,7 +640,10 @@ 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 @@ -636,6 +657,8 @@ public enum StartingStrategy { abstract @Nullable TableIdentifier getTableIdentifier(); + abstract boolean getUseCdc(); + abstract @Nullable Long getFromSnapshot(); abstract @Nullable Long getToSnapshot(); @@ -662,6 +685,8 @@ 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); @@ -683,6 +708,10 @@ abstract static class Builder { abstract ReadRows build(); } + public ReadRows withCdc() { + return toBuilder().setUseCdc(true).build(); + } + public ReadRows from(TableIdentifier tableIdentifier) { return toBuilder().setTableIdentifier(tableIdentifier).build(); } @@ -745,11 +774,12 @@ public PCollection expand(PBegin input) { .setPollInterval(getPollInterval()) .setWatermarkColumn(getWatermarkColumn()) .setWatermarkTimeUnit(getWatermarkTimeUnit()) + .setUseCdc(getUseCdc()) .build(); scanConfig.validate(table); PTransform> source = - scanConfig.useIncrementalSource() + getUseCdc() ? new IncrementalScanSource(scanConfig) : Read.from(new ScanSource(scanConfig)); 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 13febe902a9f..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 @@ -22,12 +22,10 @@ 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; @@ -39,8 +37,6 @@ 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; @@ -92,31 +88,12 @@ Row getConfigurationRow() { @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()) - .from(TableIdentifier.parse(configuration.getTable())) - .fromSnapshot(configuration.getFromSnapshot()) - .toSnapshot(configuration.getToSnapshot()) - .fromTimestamp(configuration.getFromTimestamp()) - .toTimestamp(configuration.getToTimestamp()) - .withStartingStrategy(strategy) - .withWatermarkColumn(configuration.getWatermarkColumn()) - .withWatermarkTimeUnit(configuration.getWatermarkTimeUnit()); - - PCollection output = input.getPipeline().apply(readRows); + PCollection output = + input + .getPipeline() + .apply( + IcebergIO.readRows(configuration.getIcebergCatalog()) + .from(TableIdentifier.parse(configuration.getTable()))); return PCollectionRowTuple.of(OUTPUT_TAG, output); } @@ -144,43 +121,6 @@ static Builder builder() { @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 interval at which to poll for new snapshots. Defaults to 60 seconds.") - abstract @Nullable Integer getPollIntervalSeconds(); - - @SchemaFieldDescription( - "Enables streaming reads. By default, the streaming source will start reading from the " - + "latest snapshot (inclusive) and continue polling forever based on the specified poll_interval_seconds") - abstract @Nullable Boolean getStreaming(); - - @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( - "The column used to derive event time for tracking progress. Uses the snapshot's commit timestamp by default.") - abstract @Nullable String getWatermarkColumn(); - - @SchemaFieldDescription( - "Use only when the watermark column is set to a Long type. Specifies the TimeUnit represented by the watermark column. Default is 'microseconds'. " - + "Check https://docs.oracle.com/javase/8/docs/api///?java/util/concurrent/TimeUnit.html for possible values.") - abstract @Nullable String getWatermarkTimeUnit(); - @AutoValue.Builder abstract static class Builder { abstract Builder setTable(String table); @@ -191,24 +131,6 @@ abstract static class Builder { 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 setPollIntervalSeconds(Integer pollInterval); - - abstract Builder setStreaming(Boolean streaming); - - abstract Builder setStartingStrategy(String strategy); - - abstract Builder setWatermarkColumn(String column); - - abstract Builder setWatermarkTimeUnit(String watermarkTimeUnit); - abstract Configuration 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 26a0d87cbca5..63666e78d0c9 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 @@ -23,7 +23,9 @@ import com.google.auto.value.AutoValue; import java.io.Serializable; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -49,16 +51,6 @@ @AutoValue public abstract class IcebergScanConfig implements Serializable { - public boolean useIncrementalSource() { - return MoreObjects.firstNonNull(getStreaming(), false) - || getPollInterval() != null - || getToTimestamp() != null - || getFromSnapshotInclusive() != null - || getToSnapshot() != null - || getStartingStrategy() != null - || getWatermarkColumn() != null; - } - private transient @MonotonicNonNull Table cachedTable; public enum ScanType { @@ -133,6 +125,9 @@ public Table getTable() { @Pure public abstract @Nullable StartingStrategy getStartingStrategy(); + @Pure + public abstract boolean getUseCdc(); + @Pure public abstract @Nullable Boolean getStreaming(); @@ -168,6 +163,7 @@ public static Builder builder() { .setToSnapshotRef(null) .setFromTimestamp(null) .setToTimestamp(null) + .setUseCdc(false) .setStreaming(null) .setPollInterval(null) .setStartingStrategy(null) @@ -221,6 +217,8 @@ public Builder setTableIdentifier(String... names) { 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); @@ -237,6 +235,45 @@ public Builder setTableIdentifier(String... names) { } 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 (getWatermarkColumn() != null) { + invalidOptions.add("watermark_column"); + } + if (getWatermarkTimeUnit() != null) { + invalidOptions.add("watermark_time_unit"); + } + 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, 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 index 33d4eec75c3f..22a035a811fb 100644 --- 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 @@ -47,10 +47,16 @@ import org.joda.time.Duration; /** - * An Iceberg source that reads a table incrementally using range(s) of table Snapshots. The - * unbounded implementation will continuously poll for new Snapshots at the specified frequency. A - * collection of FileScanTasks are created for each snapshot range. An SDF (shared by bounded and - * unbounded implementations) is used to process each task and output Beam rows. + * 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 @@ -70,10 +76,11 @@ public PCollection expand(PBegin input) { TableCache.getRefreshed( scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); - return MoreObjects.firstNonNull(scanConfig.getStreaming(), false) - ? readUnbounded(input).setRowSchema(ReadUtils.outputCdcSchema(table.schema())) - : readBounded(input, table) - .setRowSchema(IcebergUtils.icebergSchemaToBeamSchema(table.schema())); + PCollection rows = + MoreObjects.firstNonNull(scanConfig.getStreaming(), false) + ? readUnbounded(input) + : readBounded(input, table); + return rows.setRowSchema(ReadUtils.outputCdcSchema(table.schema())); } /** 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 index fff6e5b171d5..1f203dc4c24d 100644 --- 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 @@ -63,8 +63,8 @@ public void process( String tableIdentifier = element.getKey().getTableIdentifierString(); List readTasks = element.getValue(); Table table = TableCache.get(tableIdentifier, scanConfig.getCatalogConfig().catalog()); - Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); - Schema outputSchema = ReadUtils.outputCdcSchema(beamSchema); + Schema dataSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); + Schema outputCdcSchema = ReadUtils.outputCdcSchema(dataSchema); // SDF can split by the number of read tasks for (long taskIndex = tracker.currentRestriction().getFrom(); @@ -82,8 +82,8 @@ public void process( try (CloseableIterable reader = ReadUtils.createReader(task, table)) { for (Record record : reader) { Row row = - Row.withSchema(outputSchema) - .addValue(IcebergUtils.icebergRecordToBeamRow(beamSchema, record)) + Row.withSchema(outputCdcSchema) + .addValue(IcebergUtils.icebergRecordToBeamRow(dataSchema, record)) .addValue(operation) .build(); out.outputWithTimestamp(row, outputTimestamp); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index 63d37b750b1c..9c74db20b023 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -29,6 +29,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; +import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; @@ -52,14 +53,20 @@ public void process(@Element KV element, OutputRec String tableIdentifier = element.getKey().getTableIdentifierString(); ReadTask readTask = element.getValue(); Table table = TableCache.get(tableIdentifier, scanConfig.getCatalogConfig().catalog()); - Schema beamSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); + Schema dataSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); + Schema outputCdcSchema = ReadUtils.outputCdcSchema(dataSchema); Instant outputTimestamp = ReadUtils.getReadTaskTimestamp(readTask, scanConfig); FileScanTask task = readTask.getFileScanTask(); + @Nullable String operation = readTask.getOperation(); try (CloseableIterable reader = ReadUtils.createReader(task, table)) { for (Record record : reader) { - Row row = IcebergUtils.icebergRecordToBeamRow(beamSchema, record); + Row row = + Row.withSchema(outputCdcSchema) + .addValue(IcebergUtils.icebergRecordToBeamRow(dataSchema, record)) + .addValue(operation) + .build(); out.outputWithTimestamp(row, outputTimestamp); } } 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 index 72bcf21405b1..e92b40702c2e 100644 --- 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 @@ -107,7 +107,7 @@ public static Schema outputCdcSchema(org.apache.iceberg.Schema tableSchema) { return outputCdcSchema(IcebergUtils.icebergSchemaToBeamSchema(tableSchema)); } - public static ParquetReader createReader(FileScanTask task, Table table) { + static ParquetReader createReader(FileScanTask task, Table table) { String filePath = task.file().path().toString(); InputFile inputFile; try (FileIO io = table.io()) { 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 index 7cf3ed875941..1afce560b857 100644 --- 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 @@ -18,6 +18,7 @@ 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; @@ -64,7 +65,7 @@ public void testBuildTransformWithRow() { properties.put("warehouse", "test_location"); Row config = - Row.withSchema(new IcebergReadSchemaTransformProvider().configurationSchema()) + Row.withSchema(new IcebergCdcReadSchemaTransformProvider().configurationSchema()) .withFieldValue("table", "test_table_identifier") .withFieldValue("catalog_name", "test-name") .withFieldValue("catalog_properties", properties) @@ -78,7 +79,7 @@ public void testBuildTransformWithRow() { .withFieldValue("watermark_time_unit", "nanoseconds") .build(); - new IcebergReadSchemaTransformProvider().from(config); + new IcebergCdcReadSchemaTransformProvider().from(config); } @Test @@ -114,7 +115,7 @@ public void testSimpleScan() throws Exception { .apply(new IcebergCdcReadSchemaTransformProvider().from(readConfigBuilder.build())) .getSinglePCollection(); - assertThat(output.isBounded(), equalTo(UNBOUNDED)); + assertThat(output.isBounded(), equalTo(BOUNDED)); PAssert.that(output) .satisfies( (Iterable rows) -> { @@ -129,7 +130,7 @@ public void testSimpleScan() throws Exception { } @Test - public void testReadUsingManagedTransform() throws Exception { + public void testStreamingReadUsingManagedTransform() throws Exception { String identifier = "default.table_" + Long.toString(UUID.randomUUID().hashCode(), 16); TableIdentifier tableId = TableIdentifier.parse(identifier); @@ -149,7 +150,8 @@ public void testReadUsingManagedTransform() throws Exception { + " type: %s\n" + " warehouse: %s\n" + "from_snapshot: %s\n" - + "to_snapshot: %s", + + "to_snapshot: %s\n" + + "streaming: true", identifier, CatalogUtil.ICEBERG_CATALOG_TYPE_HADOOP, warehouse.location, second, third); final List expectedRows = 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 195634675973..e6c25a837ffb 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 @@ -21,6 +21,8 @@ import static org.apache.beam.sdk.io.iceberg.ReadUtils.RECORD; import static org.apache.beam.sdk.io.iceberg.TestFixtures.createRecord; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.iceberg.util.DateTimeUtil.microsFromTimestamp; +import static org.apache.iceberg.util.DateTimeUtil.microsToMillis; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; @@ -28,6 +30,7 @@ import java.io.File; import java.io.IOException; +import java.time.LocalDateTime; import java.util.Arrays; import java.util.List; import java.util.Map; @@ -39,10 +42,13 @@ import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; 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.transforms.SerializableFunction; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; @@ -108,17 +114,28 @@ public static Iterable data() { return Arrays.asList(new Object[][] {{false}, {true}}); } + // TODO(#34168, ahmedabu98): Update tests when we close feature gaps between regular and cdc + // sources @Parameter(0) public boolean useIncrementalScan; - static class PrintRow extends DoFn { - - @ProcessElement - public void process(@Element Row row, @Timestamp Instant timestamp, OutputReceiver output) - throws Exception { - LOG.info("Got row {}", row); - LOG.info("timestamp: " + timestamp); - output.output(row); + 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); } } @@ -128,7 +145,11 @@ public void testFailWhenBothStartingSnapshotAndTimestampAreSet() { TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); warehouse.createTable(tableId, TestFixtures.SCHEMA); IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).from(tableId).fromSnapshot(123L).fromTimestamp(123L); + IcebergIO.readRows(catalogConfig()) + .from(tableId) + .withCdc() + .fromSnapshot(123L) + .fromTimestamp(123L); thrown.expect(IllegalArgumentException.class); thrown.expectMessage( @@ -142,7 +163,11 @@ public void testFailWhenBothEndingSnapshotAndTimestampAreSet() { TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); warehouse.createTable(tableId, TestFixtures.SCHEMA); IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).from(tableId).toSnapshot(123L).toTimestamp(123L); + IcebergIO.readRows(catalogConfig()) + .withCdc() + .from(tableId) + .toSnapshot(123L) + .toTimestamp(123L); thrown.expect(IllegalArgumentException.class); thrown.expectMessage( @@ -157,6 +182,7 @@ public void testFailWhenStartingPointAndStartingStrategyAreSet() { warehouse.createTable(tableId, TestFixtures.SCHEMA); IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()) + .withCdc() .from(tableId) .fromSnapshot(123L) .withStartingStrategy(StartingStrategy.EARLIEST); @@ -174,6 +200,7 @@ public void testFailWhenPollIntervalIsSetOnBatchRead() { warehouse.createTable(tableId, TestFixtures.SCHEMA); IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()) + .withCdc() .from(tableId) .withPollInterval(Duration.standardSeconds(5)); @@ -189,7 +216,7 @@ public void testFailWhenWatermarkColumnDoesNotExist() { TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); warehouse.createTable(tableId, TestFixtures.SCHEMA); IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).from(tableId).withWatermarkColumn("unknown"); + IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).withWatermarkColumn("unknown"); thrown.expect(IllegalArgumentException.class); thrown.expectMessage( @@ -203,7 +230,7 @@ public void testFailWithInvalidWatermarkColumnType() { TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); warehouse.createTable(tableId, TestFixtures.SCHEMA); IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).from(tableId).withWatermarkColumn("data"); + IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).withWatermarkColumn("data"); thrown.expect(IllegalArgumentException.class); thrown.expectMessage( @@ -219,7 +246,7 @@ public void testFailWhenWatermarkColumnMissingMetrics() { Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); table.updateProperties().set("write.metadata.metrics.default", "none").commit(); IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).from(tableId).withWatermarkColumn("id"); + IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).withWatermarkColumn("id"); thrown.expect(IllegalStateException.class); thrown.expectMessage("Invalid source configuration: source table"); @@ -236,7 +263,7 @@ public void testFailWhenWatermarkTimeUnitUsedWithoutSpecifyingColumn() { TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); warehouse.createTable(tableId, TestFixtures.SCHEMA); IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).from(tableId).withWatermarkTimeUnit("hours"); + IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).withWatermarkTimeUnit("hours"); thrown.expect(IllegalArgumentException.class); thrown.expectMessage( @@ -257,7 +284,7 @@ public void testSimpleScan() throws Exception { IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId); if (useIncrementalScan) { - read = read.toSnapshot(simpleTable.currentSnapshot().snapshotId()); + read = read.withCdc().toSnapshot(simpleTable.currentSnapshot().snapshotId()); } final List expectedRows = expectedRecords.stream() @@ -265,8 +292,11 @@ public void testSimpleScan() throws Exception { .map(record -> IcebergUtils.icebergRecordToBeamRow(schema, record)) .collect(Collectors.toList()); - PCollection output = - testPipeline.apply(read).apply(ParDo.of(new PrintRow())).setRowSchema(schema); + PCollection output = testPipeline.apply(read).apply(new PrintRow()); + + if (useIncrementalScan) { + output = output.apply(ReadUtils.extractRecords()); + } PAssert.that(output) .satisfies( @@ -320,10 +350,13 @@ public void testIdentityColumnScan() throws Exception { IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId); if (useIncrementalScan) { - read = read.toSnapshot(simpleTable.currentSnapshot().snapshotId()); + read = read.withCdc().toSnapshot(simpleTable.currentSnapshot().snapshotId()); + } + PCollection output = testPipeline.apply(read).apply(new PrintRow()); + + if (useIncrementalScan) { + output = output.apply(ReadUtils.extractRecords()); } - PCollection output = - testPipeline.apply(read).apply(ParDo.of(new PrintRow())).setRowSchema(schema); PAssert.that(output) .satisfies( @@ -430,10 +463,9 @@ public void testNameMappingScan() throws Exception { IcebergIO.ReadRows read = IcebergIO.readRows(catalogConfig()).from(tableId); if (useIncrementalScan) { - read = read.toSnapshot(simpleTable.currentSnapshot().snapshotId()); + read = read.withCdc().toSnapshot(simpleTable.currentSnapshot().snapshotId()); } - PCollection output = - testPipeline.apply(read).apply(ParDo.of(new PrintRow())).setRowSchema(beamSchema); + PCollection output = testPipeline.apply(read).apply(new PrintRow()); final Row[] expectedRows = recordData.stream() @@ -441,6 +473,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) -> { @@ -492,7 +527,7 @@ public void testBatchReadBetweenTimestamps() throws IOException { } @Test - public void testWatermarkColumn() throws IOException { + public void testWatermarkColumnLongType() throws IOException { assumeTrue(useIncrementalScan); TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); @@ -522,6 +557,7 @@ public void testWatermarkColumn() throws IOException { IcebergIO.ReadRows readRows = IcebergIO.readRows(catalogConfig()) .from(tableId) + .withCdc() .streaming(true) .withStartingStrategy(StartingStrategy.EARLIEST) .toSnapshot(simpleTable.currentSnapshot().snapshotId()) @@ -531,9 +567,87 @@ public void testWatermarkColumn() throws IOException { output.apply( ParDo.of( new CheckWatermarks( - TestFixtures.FILE1SNAPSHOT4_DATA, - TestFixtures.FILE2SNAPSHOT4_DATA, - TestFixtures.FILE3SNAPSHOT4_DATA))); + lowestMillisOf(TestFixtures.FILE1SNAPSHOT4_DATA), + lowestMillisOf(TestFixtures.FILE2SNAPSHOT4_DATA), + lowestMillisOf(TestFixtures.FILE3SNAPSHOT4_DATA), + row -> TimeUnit.DAYS.toMillis(checkStateNotNull(row.getInt64("id")))))); + testPipeline.run().waitUntilFinish(); + } + + private static long lowestMillisOf(List> data) { + long lowestId = data.stream().mapToLong(m -> (long) m.get("id")).min().getAsLong(); + return TimeUnit.DAYS.toMillis(lowestId); + } + + @Test + public void testWatermarkColumnTimestampType() throws IOException { + assumeTrue(useIncrementalScan); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); + Schema schema = + Schema.builder() + .addStringField("str") + .addLogicalTypeField("datetime", SqlTypes.DATETIME) + .build(); + org.apache.iceberg.Schema iceSchema = IcebergUtils.beamSchemaToIcebergSchema(schema); + Table simpleTable = warehouse.createTable(tableId, iceSchema); + + LocalDateTime file1Watermark = LocalDateTime.parse("2025-01-01T12:00:00"); + List> data1 = + ImmutableList.of( + ImmutableMap.of("str", "a", "datetime", file1Watermark), + ImmutableMap.of("str", "b", "datetime", file1Watermark.plusMinutes(10)), + ImmutableMap.of("str", "c", "datetime", file1Watermark.plusMinutes(30))); + + LocalDateTime file2Watermark = LocalDateTime.parse("2025-02-01T12:00:00"); + List> data2 = + ImmutableList.of( + ImmutableMap.of("str", "d", "datetime", file2Watermark), + ImmutableMap.of("str", "e", "datetime", file2Watermark.plusMinutes(10)), + ImmutableMap.of("str", "f", "datetime", file2Watermark.plusMinutes(30))); + + LocalDateTime file3Watermark = LocalDateTime.parse("2025-03-01T12:00:00"); + List> data3 = + ImmutableList.of( + ImmutableMap.of("str", "g", "datetime", file3Watermark), + ImmutableMap.of("str", "h", "datetime", file3Watermark.plusMinutes(10)), + ImmutableMap.of("str", "i", "datetime", file3Watermark.plusMinutes(30))); + + // configure the table to capture full metrics + simpleTable + .updateProperties() + .set(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "datetime", "full") + .commit(); + + simpleTable + .newFastAppend() + .appendFile(warehouse.writeData("file1.parquet", iceSchema, data1)) + .commit(); + simpleTable + .newFastAppend() + .appendFile(warehouse.writeData("file2.parquet", iceSchema, data2)) + .appendFile(warehouse.writeData("file3.parquet", iceSchema, data3)) + .commit(); + + IcebergIO.ReadRows readRows = + IcebergIO.readRows(catalogConfig()) + .from(tableId) + .withCdc() + .streaming(true) + .withStartingStrategy(StartingStrategy.EARLIEST) + .toSnapshot(simpleTable.currentSnapshot().snapshotId()) + .withWatermarkColumn("datetime"); + PCollection output = testPipeline.apply(readRows); + output.apply( + ParDo.of( + new CheckWatermarks( + microsToMillis(microsFromTimestamp(file1Watermark)), + microsToMillis(microsFromTimestamp(file2Watermark)), + microsToMillis(microsFromTimestamp(file3Watermark)), + row -> { + LocalDateTime dt = + checkStateNotNull(row.getLogicalTypeValue("datetime", LocalDateTime.class)); + return microsToMillis(microsFromTimestamp(dt)); + }))); testPipeline.run().waitUntilFinish(); } @@ -541,21 +655,23 @@ static class CheckWatermarks extends DoFn { long file1Watermark; long file2Watermark; long file3Watermark; + SerializableFunction getMillisFn; CheckWatermarks( - List> data1, - List> data2, - List> data3) { - file1Watermark = lowestMillisOf(data1); - file2Watermark = lowestMillisOf(data2); - file3Watermark = lowestMillisOf(data3); + long file1Watermark, + long file2Watermark, + long file3Watermark, + SerializableFunction getMillisFn) { + this.file1Watermark = file1Watermark; + this.file2Watermark = file2Watermark; + this.file3Watermark = file3Watermark; + this.getMillisFn = getMillisFn; } @ProcessElement public void process(@Element Row row, @Timestamp Instant timestamp) { Row record = checkStateNotNull(row.getRow(RECORD)); - long id = checkStateNotNull(record.getInt64("id")); - long expectedMillis = TimeUnit.DAYS.toMillis(id); + long expectedMillis = getMillisFn.apply(record); long actualMillis = timestamp.getMillis(); if (expectedMillis >= file3Watermark) { @@ -566,11 +682,6 @@ public void process(@Element Row row, @Timestamp Instant timestamp) { assertEquals(file1Watermark, actualMillis); } } - - private static long lowestMillisOf(List> data) { - long lowestId = data.stream().mapToLong(m -> (long) m.get("id")).min().getAsLong(); - return TimeUnit.DAYS.toMillis(lowestId); - } } public void runWithStartingStrategy(@Nullable StartingStrategy strategy, boolean streaming) @@ -593,6 +704,7 @@ public void runWithStartingStrategy(@Nullable StartingStrategy strategy, boolean IcebergIO.ReadRows readRows = IcebergIO.readRows(catalogConfig()) .from(tableId) + .withCdc() .streaming(streaming) .toSnapshot(simpleTable.currentSnapshot().snapshotId()); if (strategy != null) { @@ -600,22 +712,20 @@ public void runWithStartingStrategy(@Nullable StartingStrategy strategy, boolean } PCollection output = testPipeline.apply(readRows); - if (streaming) { - PAssert.that(output) - .satisfies( - rows -> { - for (Row row : rows) { - assertEquals(DataOperations.APPEND, checkStateNotNull(row.getString(OPERATION))); - } - return null; - }); - output = output.apply(ReadUtils.extractRecords()); - } + 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, output.isBounded()); + assertEquals(expectedBoundedness, rows.isBounded()); - PAssert.that(output).containsInAnyOrder(expectedRows); + PAssert.that(rows).containsInAnyOrder(expectedRows); testPipeline.run().waitUntilFinish(); } @@ -640,7 +750,7 @@ public void runReadWithBoundary(boolean useSnapshotBoundary, boolean streaming) Snapshot thirdSnapshot = snapshots.get(2); IcebergIO.ReadRows readRows = - IcebergIO.readRows(catalogConfig()).from(tableId).streaming(streaming); + IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).streaming(streaming); if (useSnapshotBoundary) { readRows = @@ -652,24 +762,21 @@ public void runReadWithBoundary(boolean useSnapshotBoundary, boolean streaming) .toTimestamp(thirdSnapshot.timestampMillis() + 1); } - PCollection output = testPipeline.apply(readRows); - output = output.apply(ParDo.of(new PrintRow())).setRowSchema(output.getSchema()); - if (streaming) { - PAssert.that(output) - .satisfies( - rows -> { - for (Row row : rows) { - assertEquals(DataOperations.APPEND, checkStateNotNull(row.getString(OPERATION))); - } - return null; - }); - output = output.apply(ReadUtils.extractRecords()); - } + 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, output.isBounded()); + assertEquals(expectedBoundedness, rows.isBounded()); - PAssert.that(output).containsInAnyOrder(expectedRows); + PAssert.that(rows).containsInAnyOrder(expectedRows); testPipeline.run(); } 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 4b483b57c635..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 @@ -64,13 +64,6 @@ public void testBuildTransformWithRow() { .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("watermark_column", "abc") - .withFieldValue("watermark_time_unit", "nanoseconds") .build(); new IcebergReadSchemaTransformProvider().from(config); 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 8cb403bec1f9..22d505c676f4 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 @@ -103,6 +103,7 @@ public class IcebergSchemaTransformTranslationTest { .withFieldValue("from_snapshot", 123L) .withFieldValue("to_timestamp", 456L) .withFieldValue("poll_interval_seconds", 123) + .withFieldValue("streaming", true) .withFieldValue("watermark_column", "id") .withFieldValue("watermark_time_unit", "nanoseconds") .build(); 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 b99ac5d1bd6b..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,13 +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; @@ -108,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); 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 a59c8c48374d..9ec65c8e0754 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,6 +17,8 @@ */ 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; @@ -405,7 +407,7 @@ 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(); @@ -418,12 +420,13 @@ public void testStreamingRead() throws Exception { List expectedRows = populateTable(table); Map config = new HashMap<>(managedIcebergConfig(tableId())); + config.put("streaming", true); config.put("to_snapshot", table.currentSnapshot().snapshotId()); config.put("watermark_column", "datetime"); PCollection rows = pipeline - .apply(Managed.read(Managed.ICEBERG_CDC).withConfig(config)) + .apply(Managed.read(ICEBERG_CDC).withConfig(config)) .getSinglePCollection() .apply(ReadUtils.extractRecords()); @@ -449,9 +452,9 @@ public void testWriteRead() throws IOException { Map config = managedIcebergConfig(tableId()); pipeline - .apply("read", Managed.read(Managed.ICEBERG).withConfig(config)) + .apply("read", Managed.read(ICEBERG).withConfig(config)) .getSinglePCollection() - .apply("write", Managed.write(Managed.ICEBERG).withConfig(config)); + .apply("write", Managed.write(ICEBERG).withConfig(config)); pipeline.run().waitUntilFinish(); List returnedRecords = readRecords(table); @@ -470,15 +473,16 @@ public void testReadWriteStreaming() throws IOException { Map readConfig = new HashMap<>(config); readConfig.put("to_timestamp", System.currentTimeMillis()); readConfig.put("watermark_column", "datetime_tz"); + readConfig.put("streaming", true); Map writeConfig = new HashMap<>(config); writeConfig.put("triggering_frequency_seconds", 5); pipeline - .apply("streaming read", Managed.read(Managed.ICEBERG_CDC).withConfig(readConfig)) + .apply("streaming read", Managed.read(ICEBERG_CDC).withConfig(readConfig)) .getSinglePCollection() .apply(ReadUtils.extractRecords()) - .apply("streaming write", Managed.write(Managed.ICEBERG).withConfig(writeConfig)); + .apply("streaming write", Managed.write(ICEBERG).withConfig(writeConfig)); pipeline.run().waitUntilFinish(); List returnedRecords = readRecords(table); @@ -493,7 +497,7 @@ public void testWrite() throws IOException { // 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())); @@ -521,7 +525,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 @@ -558,7 +562,7 @@ public void testStreamingWrite() throws IOException { 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); @@ -591,7 +595,7 @@ public void testStreamingWriteWithPriorWindowing() throws IOException { 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); @@ -671,7 +675,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); @@ -754,15 +758,13 @@ public void runReadBetween(boolean useSnapshotBoundary, boolean streaming) throw config.put("from_timestamp", from.timestampMillis() - 1); config.put("to_timestamp", to.timestampMillis() + 1); } - - String source = streaming ? Managed.ICEBERG_CDC : Managed.ICEBERG; + config.put("streaming", streaming); PCollection rows = - pipeline.apply(Managed.read(source).withConfig(config)).getSinglePCollection(); - - if (streaming) { - rows = rows.apply(ReadUtils.extractRecords()); - } + pipeline + .apply(Managed.read(ICEBERG_CDC).withConfig(config)) + .getSinglePCollection() + .apply(ReadUtils.extractRecords()); IsBounded expectedBoundedness = streaming ? UNBOUNDED : BOUNDED; assertEquals(expectedBoundedness, rows.isBounded()); From 79ab16a30842166616b1037470643902177d1974 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 4 Mar 2025 15:08:46 -0500 Subject: [PATCH 23/32] add to CHANGES.md and discussion docs --- CHANGES.md | 1 + contributor-docs/discussion-docs/2025.md | 8 ++++++-- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 031e17e5edac..91b4d83282bf 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -63,6 +63,7 @@ ## I/Os * Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* [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 From 06a4cee798b1915f14cf861aba4fa99b149005e3 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 4 Mar 2025 16:05:19 -0500 Subject: [PATCH 24/32] spotless --- .../org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java | 2 +- .../main/java/org/apache/beam/sdk/io/iceberg/ReadUtils.java | 6 +----- .../java/org/apache/beam/sdk/io/iceberg/SnapshotInfo.java | 2 +- .../beam/sdk/io/iceberg/catalog/IcebergCatalogBaseIT.java | 4 ++-- 4 files changed, 5 insertions(+), 9 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java index 63666e78d0c9..ff5db1a66476 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 @@ -58,7 +58,7 @@ public enum ScanType { BATCH } - private final Set WATERMARK_COLUMN_TYPES = + private static final Set WATERMARK_COLUMN_TYPES = ImmutableSet.of( Types.LongType.get(), Types.TimestampType.withoutZone(), Types.TimestampType.withZone()); 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 index e92b40702c2e..e8e4ce8e9bd3 100644 --- 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 @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.iceberg.util.SnapshotUtil.ancestorsOf; -import com.google.common.annotations.VisibleForTesting; import java.nio.ByteBuffer; import java.util.Collection; import java.util.Collections; @@ -31,7 +30,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.BiFunction; import java.util.stream.Collectors; -import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; @@ -73,9 +71,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** Internal helper class for source operations. */ -@Internal -@VisibleForTesting +/** Helper class for source operations. */ public class ReadUtils { private static final Logger LOG = LoggerFactory.getLogger(ReadUtils.class); 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 4c67066b1926..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 @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import com.google.auto.value.AutoValue; -import com.google.common.annotations.VisibleForTesting; import java.util.Map; import javax.annotation.Nullable; import org.apache.beam.sdk.schemas.AutoValueSchema; @@ -31,6 +30,7 @@ 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; 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 9ec65c8e0754..8c297913d68d 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 @@ -248,7 +248,7 @@ public void cleanUp() throws Exception { private static final SimpleFunction ROW_FUNC = new SimpleFunction() { - final long OFFSET_2025_MILLIS = + final long offset2025Millis = LocalDate.parse("2025-01-01").toDateTime(LocalTime.MIDNIGHT).getMillis(); @Override @@ -266,7 +266,7 @@ public Row apply(Long num) { .addValue(Float.valueOf(strNum + "." + strNum)) .build(); - long timestampMillis = OFFSET_2025_MILLIS + TimeUnit.MICROSECONDS.toHours(num); + long timestampMillis = offset2025Millis + TimeUnit.MICROSECONDS.toHours(num); return Row.withSchema(BEAM_SCHEMA) .addValue("value_" + strNum) .addValue(String.valueOf((char) (97 + num % 5))) From 795c87c2984dac501f8a5dde356961a4ef488bc2 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 5 Mar 2025 10:40:27 -0500 Subject: [PATCH 25/32] address review comments about java docs --- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 46 ++++++++++--------- .../org/apache/beam/sdk/managed/Managed.java | 1 + 2 files changed, 25 insertions(+), 22 deletions(-) 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 010cb5d48545..568be8dd9656 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 @@ -142,6 +142,20 @@ * 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). @@ -150,8 +164,8 @@ * * {@code to_snapshot} * {@code long} - * Reads up to this snapshot ID (inclusive). If unset and the source is bounded, it will read - * up to the current snapshot (inclusive). If unset and source is unbounded, it will continue polling for new snapshots forever. + * 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. * * * @@ -163,7 +177,8 @@ * * {@code to_timestamp} * {@code long} - * Reads up to the latest snapshot (inclusive) created before this timestamp (in milliseconds). + * 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. * * * @@ -182,7 +197,7 @@ * {@code watermark_column} * {@code str} * - * The column used to derive event time to track progress. Must be of type: + * The column used to derive event time to track progress. Must be one of the following Iceberg types: *
    *
  • {@code timestamp}
  • *
  • {@code timestamptz}
  • @@ -200,20 +215,6 @@ *

    Check {@link TimeUnit} for possible values. * * - * - * {@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. - * - * * * *

    Beam Rows

    @@ -476,7 +477,7 @@ * {@code operation} * {@code string} * - * The snapshot operation associated with this record (e.g. "append", "replace", "delete") + * The snapshot operation associated with this record. For now, only "append" is supported. * * * @@ -513,8 +514,8 @@ *

    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 also be overridden with a - * couple of mutually exclusive options: + * 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}. @@ -537,7 +538,8 @@ * .getSinglePCollection(); * } * - * Note: An end point can also be set when performing a streaming read. + * 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. * *

      Handling Watermarks (CDC only)

      * 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 4d5a3567bff6..9f676cc05f3d 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 @@ -110,6 +110,7 @@ public class Managed { * *
        *
      • {@link Managed#ICEBERG} : Read from Apache Iceberg tables + *
      • {@link Managed#ICEBERG_CDC} : CDC Read from Apache Iceberg tables *
      • {@link Managed#KAFKA} : Read from Apache Kafka topics *
      • {@link Managed#BIGQUERY} : Read from GCP BigQuery tables *
      From c6461c9194753afc112001c9d906b3e0eae1d1f7 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 5 Mar 2025 10:42:16 -0500 Subject: [PATCH 26/32] remove raw guava dep --- sdks/java/io/iceberg/build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index e904f4921721..32c95ea8260c 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -44,7 +44,6 @@ def hive_version = "3.1.3" dependencies { implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.guava implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":model:pipeline", configuration: "shadow") implementation library.java.avro From 7dbf3e1736c0d501f571ad7ffff672b3c1823158 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 5 Mar 2025 13:36:38 -0500 Subject: [PATCH 27/32] add another test for read utils --- .../sdk/io/iceberg/IcebergScanConfig.java | 4 + .../apache/beam/sdk/io/iceberg/ReadUtils.java | 2 - .../beam/sdk/io/iceberg/ReadUtilsTest.java | 123 +++++++++++++++++- 3 files changed, 122 insertions(+), 7 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java index ff5db1a66476..1a40b1da7b25 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java @@ -31,6 +31,7 @@ import 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.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.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; @@ -234,6 +235,9 @@ public Builder setTableIdentifier(String... names) { 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()) { 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 index e8e4ce8e9bd3..1ec115aaedb1 100644 --- 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 @@ -263,10 +263,8 @@ static Instant getReadTaskTimestamp(ReadTask readTask, IcebergScanConfig scanCon ReadUtils.getLowerBoundTimestampMillis( readTask.getFileScanTask(), watermarkColumn, scanConfig.getWatermarkTimeUnit()), BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); - System.out.println("xxx using file millis " + millis); } else { millis = readTask.getSnapshotTimestampMillis(); - System.out.println("xxx using snapshot millis " + millis); } return Instant.ofEpochMilli(millis); } 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 index 107224926969..5efe6cd76a72 100644 --- 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 @@ -17,13 +17,17 @@ */ 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.UUID; +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; @@ -37,21 +41,23 @@ 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", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); Map> data = @@ -89,8 +95,7 @@ public void testCreateReader() throws IOException { @Test public void testSnapshotsBetween() throws IOException { - TableIdentifier tableId = - TableIdentifier.of("default", "table" + Long.toString(UUID.randomUUID().hashCode(), 16)); + TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); Map> data = @@ -125,4 +130,112 @@ public void testSnapshotsBetween() throws IOException { .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); + } + } } From 40fe4ab3ecd5f072ed60a2c3059d8fb00a20cffe Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Fri, 7 Mar 2025 10:56:22 -0500 Subject: [PATCH 28/32] use cached schemas --- .../apache/beam/sdk/io/iceberg/IcebergScanConfig.java | 9 +++++++++ .../beam/sdk/io/iceberg/ReadFromGroupedTasks.java | 11 ++++------- .../org/apache/beam/sdk/io/iceberg/ReadFromTasks.java | 11 ++++------- 3 files changed, 17 insertions(+), 14 deletions(-) diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java index 1a40b1da7b25..49de022b2f22 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 @@ -53,6 +53,7 @@ @AutoValue public abstract class IcebergScanConfig implements Serializable { private transient @MonotonicNonNull Table cachedTable; + private @MonotonicNonNull Schema cachedCdcSchema; public enum ScanType { TABLE, @@ -84,6 +85,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(); 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 index 1f203dc4c24d..74b3f6d6be31 100644 --- 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 @@ -23,7 +23,6 @@ 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.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.values.KV; @@ -60,11 +59,9 @@ public void process( RestrictionTracker tracker, OutputReceiver out) throws IOException, ExecutionException { - String tableIdentifier = element.getKey().getTableIdentifierString(); List readTasks = element.getValue(); - Table table = TableCache.get(tableIdentifier, scanConfig.getCatalogConfig().catalog()); - Schema dataSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); - Schema outputCdcSchema = ReadUtils.outputCdcSchema(dataSchema); + Table table = + TableCache.get(scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); // SDF can split by the number of read tasks for (long taskIndex = tracker.currentRestriction().getFrom(); @@ -82,8 +79,8 @@ public void process( try (CloseableIterable reader = ReadUtils.createReader(task, table)) { for (Record record : reader) { Row row = - Row.withSchema(outputCdcSchema) - .addValue(IcebergUtils.icebergRecordToBeamRow(dataSchema, record)) + Row.withSchema(scanConfig.getCdcSchema()) + .addValue(IcebergUtils.icebergRecordToBeamRow(scanConfig.getSchema(), record)) .addValue(operation) .build(); out.outputWithTimestamp(row, outputTimestamp); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index 9c74db20b023..dd64b27c52d6 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -21,7 +21,6 @@ import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; @@ -50,11 +49,9 @@ class ReadFromTasks extends DoFn, Row> { @ProcessElement public void process(@Element KV element, OutputReceiver out) throws IOException, ExecutionException { - String tableIdentifier = element.getKey().getTableIdentifierString(); ReadTask readTask = element.getValue(); - Table table = TableCache.get(tableIdentifier, scanConfig.getCatalogConfig().catalog()); - Schema dataSchema = IcebergUtils.icebergSchemaToBeamSchema(table.schema()); - Schema outputCdcSchema = ReadUtils.outputCdcSchema(dataSchema); + Table table = + TableCache.get(scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); Instant outputTimestamp = ReadUtils.getReadTaskTimestamp(readTask, scanConfig); FileScanTask task = readTask.getFileScanTask(); @@ -63,8 +60,8 @@ public void process(@Element KV element, OutputRec try (CloseableIterable reader = ReadUtils.createReader(task, table)) { for (Record record : reader) { Row row = - Row.withSchema(outputCdcSchema) - .addValue(IcebergUtils.icebergRecordToBeamRow(dataSchema, record)) + Row.withSchema(scanConfig.getCdcSchema()) + .addValue(IcebergUtils.icebergRecordToBeamRow(scanConfig.getSchema(), record)) .addValue(operation) .build(); out.outputWithTimestamp(row, outputTimestamp); From db3c5704dd4743e8b680e5482ed085c287fc5d4c Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 10 Mar 2025 14:20:47 -0400 Subject: [PATCH 29/32] watermark based on snapshot timestamp; remove infinite allowed skew; remove window step; add --strea ming=true validation; add IO links to Managed java doc --- .../sdk/io/iceberg/CreateReadTasksDoFn.java | 5 ++++- .../sdk/io/iceberg/IncrementalScanSource.java | 14 ++++++------- .../sdk/io/iceberg/ReadFromGroupedTasks.java | 8 ------- .../beam/sdk/io/iceberg/ReadFromTasks.java | 8 ------- .../sdk/io/iceberg/WatchForSnapshots.java | 7 +++++-- .../org/apache/beam/sdk/managed/Managed.java | 21 ++++++++++++------- 6 files changed, 29 insertions(+), 34 deletions(-) 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 index ad7b65d4e05b..b8ddf4df40e7 100644 --- 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 @@ -24,6 +24,7 @@ import java.util.List; import java.util.concurrent.ExecutionException; 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.DoFn; import org.apache.beam.sdk.values.KV; @@ -35,6 +36,7 @@ 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; @@ -113,7 +115,8 @@ private void createAndOutputReadTasks( .setTableIdentifierString(checkStateNotNull(snapshot.getTableIdentifierString())) .build(); - out.output(KV.of(descriptor, task)); + out.outputWithTimestamp( + KV.of(descriptor, task), Instant.ofEpochMilli(snapshot.getTimestampMillis())); totalScanTasks.inc(); } } 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 index 22a035a811fb..208102997a9b 100644 --- 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 @@ -24,6 +24,7 @@ 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.options.StreamingOptions; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.GroupIntoBatches; import org.apache.beam.sdk.transforms.MapElements; @@ -31,16 +32,13 @@ 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.transforms.windowing.AfterPane; -import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -import org.apache.beam.sdk.transforms.windowing.Repeatedly; -import org.apache.beam.sdk.transforms.windowing.Window; 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.base.Preconditions; 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; @@ -97,6 +95,9 @@ public PCollection expand(PBegin input) { *
    */ private PCollection readUnbounded(PBegin input) { + Preconditions.checkState( + input.getPipeline().getOptions().as(StreamingOptions.class).isStreaming(), + "The streaming source requires '--streaming=true', but found '--streaming=false'."); @Nullable Duration pollInterval = MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); @@ -105,10 +106,6 @@ private PCollection readUnbounded(PBegin input) { .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( - Window.>into(new GlobalWindows()) - .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1))) - .discardingFiredPanes()) .apply( GroupIntoBatches.ofByteSize( MAX_FILES_BATCH_BYTE_SIZE, ReadTask::getByteSize) @@ -119,6 +116,7 @@ private PCollection readUnbounded(PBegin input) { ShardedKey.Coder.of(ReadTaskDescriptor.getCoder()), IterableCoder.of(ReadTask.getCoder()))) .apply( + "Iterable to List", MapElements.via( new SimpleFunction< KV, Iterable>, 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 index 74b3f6d6be31..7843accfb2b3 100644 --- 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 @@ -32,7 +32,6 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Duration; import org.joda.time.Instant; /** @@ -106,11 +105,4 @@ public double getSize( } return size; } - - // infinite skew in case we encounter some files that don't support watermark column statistics, - // in which case we output a -inf timestamp. - @Override - public Duration getAllowedTimestampSkew() { - return Duration.millis(Long.MAX_VALUE); - } } diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index dd64b27c52d6..b8291eb2630b 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -29,7 +29,6 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Duration; import org.joda.time.Instant; /** @@ -69,11 +68,4 @@ public void process(@Element KV element, OutputRec } scanTasksCompleted.inc(); } - - // infinite skew in case we encounter some files that don't support watermark column statistics, - // in which case we output a -inf timestamp. - @Override - public Duration getAllowedTimestampSkew() { - return Duration.millis(Long.MAX_VALUE); - } } 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 index 5ca124b8d4ba..7c93e4c095b6 100644 --- 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 @@ -27,17 +27,18 @@ 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.transforms.windowing.BoundedWindow; 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.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; 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; /** * Keeps watch over an Iceberg table and continuously outputs a range of snapshots, at the specified @@ -106,7 +107,9 @@ private PollResult> getPollResult( @Nullable List snapshots, boolean isComplete) { List>> timestampedSnapshots = new ArrayList<>(1); if (snapshots != null) { - timestampedSnapshots.add(TimestampedValue.of(snapshots, BoundedWindow.TIMESTAMP_MIN_VALUE)); + // watermark based on the most recent snapshot timestamp + Instant watermark = Instant.ofEpochMilli(Iterables.getLast(snapshots).getTimestampMillis()); + timestampedSnapshots.add(TimestampedValue.of(snapshots, watermark)); } return isComplete 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 b85ab40fa4b5..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 @@ -116,10 +116,14 @@ public class Managed { * supported managed sources are: * *
      - *
    • {@link Managed#ICEBERG} : Read from Apache Iceberg tables - *
    • {@link Managed#ICEBERG_CDC} : CDC 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) { @@ -139,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) { From 7078f20a9c906e345a5d5b72296b806cf06db919 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Mon, 10 Mar 2025 15:12:36 -0400 Subject: [PATCH 30/32] remove check --- .../org/apache/beam/sdk/io/iceberg/CreateReadTasksDoFn.java | 1 - .../apache/beam/sdk/io/iceberg/IncrementalScanSource.java | 5 ----- .../org/apache/beam/sdk/io/iceberg/WatchForSnapshots.java | 5 ++--- 3 files changed, 2 insertions(+), 9 deletions(-) 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 index b8ddf4df40e7..963630cd595a 100644 --- 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 @@ -24,7 +24,6 @@ import java.util.List; import java.util.concurrent.ExecutionException; 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.DoFn; import org.apache.beam.sdk.values.KV; 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 index 208102997a9b..21f8e685a8d7 100644 --- 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 @@ -24,7 +24,6 @@ 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.options.StreamingOptions; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.GroupIntoBatches; import org.apache.beam.sdk.transforms.MapElements; @@ -38,7 +37,6 @@ 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.iceberg.Table; import org.checkerframework.checker.nullness.qual.Nullable; @@ -95,9 +93,6 @@ public PCollection expand(PBegin input) { *
*/ private PCollection readUnbounded(PBegin input) { - Preconditions.checkState( - input.getPipeline().getOptions().as(StreamingOptions.class).isStreaming(), - "The streaming source requires '--streaming=true', but found '--streaming=false'."); @Nullable Duration pollInterval = MoreObjects.firstNonNull(scanConfig.getPollInterval(), DEFAULT_POLL_INTERVAL); 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 index 7c93e4c095b6..ffa2104ded00 100644 --- 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 @@ -33,7 +33,6 @@ 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.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.checkerframework.checker.nullness.qual.Nullable; @@ -107,8 +106,8 @@ private PollResult> getPollResult( @Nullable List snapshots, boolean isComplete) { List>> timestampedSnapshots = new ArrayList<>(1); if (snapshots != null) { - // watermark based on the most recent snapshot timestamp - Instant watermark = Instant.ofEpochMilli(Iterables.getLast(snapshots).getTimestampMillis()); + // 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)); } From fce87dc48193d4962eb5554bf1392d0080e727c5 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Tue, 11 Mar 2025 12:48:15 -0400 Subject: [PATCH 31/32] remove watermark support; add a counter and some helpful logging --- .../sdk/io/iceberg/CreateReadTasksDoFn.java | 14 +- ...IcebergCdcReadSchemaTransformProvider.java | 15 -- .../apache/beam/sdk/io/iceberg/IcebergIO.java | 59 ----- .../sdk/io/iceberg/IcebergScanConfig.java | 87 ------- .../sdk/io/iceberg/IncrementalScanSource.java | 1 + .../sdk/io/iceberg/ReadFromGroupedTasks.java | 4 +- .../beam/sdk/io/iceberg/ReadFromTasks.java | 4 +- .../apache/beam/sdk/io/iceberg/ReadUtils.java | 62 +---- .../sdk/io/iceberg/WatchForSnapshots.java | 13 + ...ergCdcReadSchemaTransformProviderTest.java | 2 - .../sdk/io/iceberg/IcebergIOReadTest.java | 228 ------------------ ...IcebergSchemaTransformTranslationTest.java | 2 - .../iceberg/catalog/IcebergCatalogBaseIT.java | 4 - 13 files changed, 27 insertions(+), 468 deletions(-) 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 index 963630cd595a..751610c599ef 100644 --- 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 @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.concurrent.ExecutionException; import org.apache.beam.sdk.metrics.Counter; @@ -52,11 +51,9 @@ class CreateReadTasksDoFn // number? private final IcebergScanConfig scanConfig; - private final @Nullable String watermarkColumnName; CreateReadTasksDoFn(IcebergScanConfig scanConfig) { this.scanConfig = scanConfig; - this.watermarkColumnName = scanConfig.getWatermarkColumn(); } @ProcessElement @@ -64,7 +61,8 @@ public void process( @Element KV> element, OutputReceiver> out) throws IOException, ExecutionException { - Table table = TableCache.get(element.getKey(), scanConfig.getCatalogConfig().catalog()); + Table table = + TableCache.getRefreshed(element.getKey(), scanConfig.getCatalogConfig().catalog()); List snapshots = element.getValue(); // scan snapshots individually and assign commit timestamp to files @@ -80,14 +78,11 @@ public void process( snapshot.getSnapshotId()); } - LOG.info("Planning to scan snapshot id range ({}, {}]", fromSnapshot, toSnapshot); + LOG.info("Planning to scan snapshot {}", toSnapshot); IncrementalAppendScan scan = table.newIncrementalAppendScan().toSnapshot(toSnapshot); if (fromSnapshot != null) { scan = scan.fromSnapshotExclusive(fromSnapshot); } - if (watermarkColumnName != null) { - scan = scan.includeColumnStats(Collections.singletonList(watermarkColumnName)); - } createAndOutputReadTasks(scan, snapshot, out); } @@ -98,6 +93,7 @@ private void createAndOutputReadTasks( 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 @@ -117,8 +113,10 @@ private void createAndOutputReadTasks( 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/IcebergCdcReadSchemaTransformProvider.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergCdcReadSchemaTransformProvider.java index cf16d874038c..0064b49475d0 100644 --- 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 @@ -115,8 +115,6 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { .fromTimestamp(configuration.getFromTimestamp()) .toTimestamp(configuration.getToTimestamp()) .withStartingStrategy(strategy) - .withWatermarkColumn(configuration.getWatermarkColumn()) - .withWatermarkTimeUnit(configuration.getWatermarkTimeUnit()) .streaming(configuration.getStreaming()); @Nullable Integer pollIntervalSeconds = configuration.getPollIntervalSeconds(); @@ -171,15 +169,6 @@ static Builder builder() { + "by setting a starting snapshot or timestamp. Defaults to earliest for batch, and latest for streaming.") abstract @Nullable String getStartingStrategy(); - @SchemaFieldDescription( - "The column used to derive event time for tracking progress. Uses the snapshot's commit timestamp by default.") - abstract @Nullable String getWatermarkColumn(); - - @SchemaFieldDescription( - "Use only when the watermark column is set to a Long type. Specifies the TimeUnit represented by the watermark column. Default is 'microseconds'. " - + "Check https://docs.oracle.com/javase/8/docs/api///?java/util/concurrent/TimeUnit.html for possible values.") - abstract @Nullable String getWatermarkTimeUnit(); - @SchemaFieldDescription( "Enables streaming reads, where source continuously polls for snapshots forever.") abstract @Nullable Boolean getStreaming(); @@ -208,10 +197,6 @@ abstract static class Builder { abstract Builder setStartingStrategy(String strategy); - abstract Builder setWatermarkColumn(String column); - - abstract Builder setWatermarkTimeUnit(String watermarkTimeUnit); - abstract Builder setPollIntervalSeconds(Integer pollInterval); abstract Builder setStreaming(Boolean streaming); 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 568be8dd9656..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 @@ -22,7 +22,6 @@ import com.google.auto.value.AutoValue; import java.util.Arrays; import java.util.List; -import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.schemas.Schema; @@ -193,28 +192,6 @@ *

Defaults to {@code earliest} for batch, and {@code latest} for streaming. * * - * - * {@code watermark_column} - * {@code str} - * - * The column used to derive event time to track progress. Must be one of the following Iceberg types: - *

    - *
  • {@code timestamp}
  • - *
  • {@code timestamptz}
  • - *
  • {@code long} (micros)
  • - *
- * - * - * - * {@code watermark_time_unit} - * {@code str} - * - * Use only when {@code watermark_column} is set to a column of type Long. Specifies the TimeUnit represented by the watermark column. - * Default is {@code "microseconds"}. - * - *

Check {@link TimeUnit} for possible values. - * - * * * *

Beam Rows

@@ -540,24 +517,6 @@ * * 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. - * - *

Handling Watermarks (CDC only)

- * - * By default, a snapshot's commit timestamp is assigned to all the records it contains. - * - *

For greater watermark precision, specify a {@code watermark_column}, which allows the - * source to extract the lower bound value from each data file and assign it to the corresponding - * records. For Long column types, you can also specify the TimeUnit represented by the column: - * - *

{@code
- * config.put("watermark_column", "flight_arrival");
- * config.put("watermark_time_unit", "hours");
- * }
- * - * Note: this requires the data files to have been written with metrics enabled. For more - * details, refer to the write - * properties. */ @Internal public class IcebergIO { @@ -675,10 +634,6 @@ public enum StartingStrategy { abstract @Nullable Duration getPollInterval(); - abstract @Nullable String getWatermarkColumn(); - - abstract @Nullable String getWatermarkTimeUnit(); - abstract Builder toBuilder(); @AutoValue.Builder @@ -703,10 +658,6 @@ abstract static class Builder { abstract Builder setPollInterval(@Nullable Duration triggeringFrequency); - abstract Builder setWatermarkColumn(@Nullable String column); - - abstract Builder setWatermarkTimeUnit(@Nullable String timeUnit); - abstract ReadRows build(); } @@ -746,14 +697,6 @@ public ReadRows withStartingStrategy(@Nullable StartingStrategy strategy) { return toBuilder().setStartingStrategy(strategy).build(); } - public ReadRows withWatermarkColumn(@Nullable String column) { - return toBuilder().setWatermarkColumn(column).build(); - } - - public ReadRows withWatermarkTimeUnit(@Nullable String timeUnit) { - return toBuilder().setWatermarkTimeUnit(timeUnit).build(); - } - @Override public PCollection expand(PBegin input) { TableIdentifier tableId = @@ -774,8 +717,6 @@ public PCollection expand(PBegin input) { .setStartingStrategy(getStartingStrategy()) .setStreaming(getStreaming()) .setPollInterval(getPollInterval()) - .setWatermarkColumn(getWatermarkColumn()) - .setWatermarkTimeUnit(getWatermarkTimeUnit()) .setUseCdc(getUseCdc()) .build(); scanConfig.validate(table); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/IcebergScanConfig.java index 49de022b2f22..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,34 +17,20 @@ */ package org.apache.beam.sdk.io.iceberg; -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; import java.io.Serializable; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; -import java.util.Set; -import java.util.concurrent.TimeUnit; -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.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.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.MetricsConfig; -import org.apache.iceberg.MetricsModes; -import org.apache.iceberg.MetricsModes.Full; -import org.apache.iceberg.MetricsModes.Truncate; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.types.Types.NestedField; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; import org.checkerframework.checker.nullness.qual.Nullable; import org.checkerframework.dataflow.qual.Pure; @@ -60,10 +46,6 @@ public enum ScanType { BATCH } - private static final Set WATERMARK_COLUMN_TYPES = - ImmutableSet.of( - Types.LongType.get(), Types.TimestampType.withoutZone(), Types.TimestampType.withZone()); - @Pure public abstract ScanType getScanType(); @@ -144,12 +126,6 @@ public Schema getCdcSchema() { @Pure public abstract @Nullable Duration getPollInterval(); - @Pure - public abstract @Nullable String getWatermarkColumn(); - - @Pure - public abstract @Nullable String getWatermarkTimeUnit(); - @Pure public abstract @Nullable String getTag(); @@ -233,10 +209,6 @@ public Builder setTableIdentifier(String... names) { public abstract Builder setPollInterval(@Nullable Duration pollInterval); - public abstract Builder setWatermarkColumn(@Nullable String column); - - public abstract Builder setWatermarkTimeUnit(@Nullable String timeUnit); - public abstract Builder setTag(@Nullable String tag); public abstract Builder setBranch(@Nullable String branch); @@ -272,12 +244,6 @@ void validate(Table table) { if (getStartingStrategy() != null) { invalidOptions.add("starting_strategy"); } - if (getWatermarkColumn() != null) { - invalidOptions.add("watermark_column"); - } - if (getWatermarkTimeUnit() != null) { - invalidOptions.add("watermark_time_unit"); - } if (!invalidOptions.isEmpty()) { throw new IllegalArgumentException( error( @@ -305,59 +271,6 @@ void validate(Table table) { Boolean.TRUE.equals(getStreaming()), error("'poll_interval_seconds' can only be set when streaming is true")); } - - @Nullable String watermarkColumn = getWatermarkColumn(); - if (watermarkColumn != null) { - org.apache.iceberg.Schema icebergSchema = IcebergUtils.beamSchemaToIcebergSchema(getSchema()); - NestedField field = - checkArgumentNotNull( - icebergSchema.findField(watermarkColumn), - error("the specified 'watermark_column' field does not exist: '%s'."), - watermarkColumn); - - Type type = field.type(); - checkArgument( - WATERMARK_COLUMN_TYPES.contains(type), - error("invalid 'watermark_column' type: %s. Valid types are %s."), - type, - WATERMARK_COLUMN_TYPES); - - MetricsModes.MetricsMode mode = MetricsConfig.forTable(table).columnMode(watermarkColumn); - checkState( - mode instanceof Truncate || mode instanceof Full, - error( - "source table '%s' is not configured to capture lower bound metrics for the specified watermark column '%s'. " - + "Valid metric modes are '%s', but found '%s'. See " - + "table option 'write.metadata.metrics...'in " - + "https://iceberg.apache.org/docs/latest/configuration/#write-properties for more information."), - getTableIdentifier(), - watermarkColumn, - ImmutableSet.of("truncate(n)", "full"), - mode); - - @Nullable String watermarkTimeUnit = getWatermarkTimeUnit(); - if (watermarkTimeUnit != null) { - checkArgument( - type.equals(Types.LongType.get()), - error( - "'watermark_time_unit' is only applicable for Long types. The specified " - + "'watermark_column' is of type: %s"), - type); - - Set validTimeUnits = - Arrays.stream(TimeUnit.values()).map(TimeUnit::name).collect(Collectors.toSet()); - checkArgument( - validTimeUnits.contains(watermarkTimeUnit.toUpperCase()), - error("invalid 'watermark_time_unit': %s. Please choose one of: %s"), - watermarkTimeUnit, - validTimeUnits); - } - } else { - checkArgument( - getWatermarkTimeUnit() == null, - error( - "cannot set 'watermark_time_unit' without also setting a 'watermark_column' of Long type.")); - } } private String error(String message) { 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 index 21f8e685a8d7..d3e638b067a9 100644 --- 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 @@ -99,6 +99,7 @@ private PCollection readUnbounded(PBegin input) { 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( 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 index 7843accfb2b3..3df7891e9b07 100644 --- 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 @@ -32,7 +32,6 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Instant; /** * Unbounded read implementation. @@ -73,7 +72,6 @@ public void process( ReadTask readTask = readTasks.get((int) taskIndex); @Nullable String operation = readTask.getOperation(); FileScanTask task = readTask.getFileScanTask(); - Instant outputTimestamp = ReadUtils.getReadTaskTimestamp(readTask, scanConfig); try (CloseableIterable reader = ReadUtils.createReader(task, table)) { for (Record record : reader) { @@ -82,7 +80,7 @@ public void process( .addValue(IcebergUtils.icebergRecordToBeamRow(scanConfig.getSchema(), record)) .addValue(operation) .build(); - out.outputWithTimestamp(row, outputTimestamp); + out.output(row); } } scanTasksCompleted.inc(); diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java index b8291eb2630b..d85f4202a6e1 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/ReadFromTasks.java @@ -29,7 +29,6 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.io.CloseableIterable; import org.checkerframework.checker.nullness.qual.Nullable; -import org.joda.time.Instant; /** * Bounded read implementation. @@ -52,7 +51,6 @@ public void process(@Element KV element, OutputRec Table table = TableCache.get(scanConfig.getTableIdentifier(), scanConfig.getCatalogConfig().catalog()); - Instant outputTimestamp = ReadUtils.getReadTaskTimestamp(readTask, scanConfig); FileScanTask task = readTask.getFileScanTask(); @Nullable String operation = readTask.getOperation(); @@ -63,7 +61,7 @@ public void process(@Element KV element, OutputRec .addValue(IcebergUtils.icebergRecordToBeamRow(scanConfig.getSchema(), record)) .addValue(operation) .build(); - out.outputWithTimestamp(row, outputTimestamp); + out.output(row); } } scanTasksCompleted.inc(); 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 index 1ec115aaedb1..2516e4db76c3 100644 --- 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 @@ -20,14 +20,12 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.iceberg.util.SnapshotUtil.ancestorsOf; -import java.nio.ByteBuffer; 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.concurrent.TimeUnit; import java.util.function.BiFunction; import java.util.stream.Collectors; import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; @@ -35,7 +33,6 @@ 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.transforms.windowing.BoundedWindow; 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; @@ -45,6 +42,7 @@ 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; @@ -58,23 +56,17 @@ import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.parquet.ParquetReader; -import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; 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; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** Helper class for source operations. */ public class ReadUtils { - private static final Logger LOG = LoggerFactory.getLogger(ReadUtils.class); - // 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 = @@ -85,7 +77,6 @@ public class ReadUtils { "parquet.crypto.factory.class"); static final String OPERATION = "operation"; static final String RECORD = "record"; - static final String DEFAULT_WATERMARK_TIME_UNIT = TimeUnit.MICROSECONDS.name(); /** Extracts {@link Row}s after a CDC streaming read. */ public static PTransform, PCollection> extractRecords() { @@ -177,7 +168,10 @@ static ParquetReader createReader(FileScanTask task, Table table) { } // 3. get current snapshot if starting_strategy is LATEST if (fromSnapshot == null && startingStrategy.equals(StartingStrategy.LATEST)) { - fromSnapshot = table.currentSnapshot().snapshotId(); + @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. @@ -223,52 +217,6 @@ static List snapshotsBetween( return snapshotIds; } - static @Nullable Long getLowerBoundTimestampMillis( - FileScanTask fileScanTask, String watermarkColumnName, @Nullable String watermarkTimeUnit) { - Types.NestedField watermarkColumn = fileScanTask.schema().findField(watermarkColumnName); - int watermarkColumnId = watermarkColumn.fieldId(); - @Nullable Map lowerBounds = fileScanTask.file().lowerBounds(); - - if (lowerBounds != null && lowerBounds.containsKey(watermarkColumnId)) { - TimeUnit timeUnit = - TimeUnit.valueOf( - MoreObjects.firstNonNull(watermarkTimeUnit, DEFAULT_WATERMARK_TIME_UNIT) - .toUpperCase()); - return timeUnit.toMillis( - Conversions.fromByteBuffer( - Types.LongType.get(), checkStateNotNull(lowerBounds.get(watermarkColumnId)))); - } else { - LOG.warn( - "Could not find statistics for watermark column '{}' in file '{}'.", - watermarkColumnName, - fileScanTask.file().path()); - return null; - } - } - - /** - * Returns the output timestamp associated with this read task. - * - *

If a watermark column is not specified, we fall back on the snapshot's commit timestamp. - * - *

If a watermark column is specified, we attempt to fetch it from the file's stats. If that - * information isn't available for whatever reason, we default to -inf. - */ - static Instant getReadTaskTimestamp(ReadTask readTask, IcebergScanConfig scanConfig) { - long millis; - @Nullable String watermarkColumn = scanConfig.getWatermarkColumn(); - if (watermarkColumn != null) { - millis = - MoreObjects.firstNonNull( - ReadUtils.getLowerBoundTimestampMillis( - readTask.getFileScanTask(), watermarkColumn, scanConfig.getWatermarkTimeUnit()), - BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); - } else { - millis = readTask.getSnapshotTimestampMillis(); - } - return Instant.ofEpochMilli(millis); - } - private static class ExtractRecords extends PTransform, PCollection> { @Override public PCollection expand(PCollection input) { 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 index ffa2104ded00..8577306f4226 100644 --- 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 @@ -21,7 +21,9 @@ 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; @@ -38,6 +40,8 @@ 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 @@ -46,6 +50,7 @@ *

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; @@ -67,6 +72,8 @@ public PCollection>> expand(PBegin input) { private static class SnapshotPollFn extends Watch.Growth.PollFn> { private final Gauge latestSnapshot = Metrics.gauge(SnapshotPollFn.class, "latestSnapshot"); + private final Counter totalSnapshotsObserved = + Metrics.counter(SnapshotPollFn.class, "numSnapshotsObserved"); private final IcebergScanConfig scanConfig; private @Nullable Long fromSnapshotId; @@ -98,6 +105,7 @@ public PollResult> apply(String tableIdentifier, Context c) { List snapshots = ReadUtils.snapshotsBetween(table, tableIdentifier, fromSnapshotId, toSnapshotId); + fromSnapshotId = currentSnapshotId; return getPollResult(snapshots, isComplete); } @@ -109,6 +117,11 @@ private PollResult> getPollResult( // 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())); + totalSnapshotsObserved.inc(snapshots.size()); } return isComplete 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 index 1afce560b857..ff00da8d609d 100644 --- 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 @@ -75,8 +75,6 @@ public void testBuildTransformWithRow() { .withFieldValue("to_timestamp", 456L) .withFieldValue("starting_strategy", "earliest") .withFieldValue("poll_interval_seconds", 789) - .withFieldValue("watermark_column", "abc") - .withFieldValue("watermark_time_unit", "nanoseconds") .build(); new IcebergCdcReadSchemaTransformProvider().from(config); 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 e6c25a837ffb..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 @@ -18,11 +18,8 @@ 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.ReadUtils.RECORD; import static org.apache.beam.sdk.io.iceberg.TestFixtures.createRecord; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; -import static org.apache.iceberg.util.DateTimeUtil.microsFromTimestamp; -import static org.apache.iceberg.util.DateTimeUtil.microsToMillis; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; @@ -30,25 +27,21 @@ import java.io.File; import java.io.IOException; -import java.time.LocalDateTime; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.UUID; -import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.io.iceberg.IcebergIO.ReadRows.StartingStrategy; import org.apache.beam.sdk.schemas.Schema; -import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; 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.transforms.SerializableFunction; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; @@ -82,7 +75,6 @@ import org.apache.parquet.hadoop.ParquetWriter; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Duration; -import org.joda.time.Instant; import org.junit.ClassRule; import org.junit.Rule; import org.junit.Test; @@ -210,68 +202,6 @@ public void testFailWhenPollIntervalIsSetOnBatchRead() { read.expand(PBegin.in(testPipeline)); } - @Test - public void testFailWhenWatermarkColumnDoesNotExist() { - assumeTrue(useIncrementalScan); - TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - warehouse.createTable(tableId, TestFixtures.SCHEMA); - IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).withWatermarkColumn("unknown"); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage( - "Invalid source configuration: the specified 'watermark_column' field does not exist: 'unknown'"); - read.expand(PBegin.in(testPipeline)); - } - - @Test - public void testFailWithInvalidWatermarkColumnType() { - assumeTrue(useIncrementalScan); - TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - warehouse.createTable(tableId, TestFixtures.SCHEMA); - IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).withWatermarkColumn("data"); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage( - "Invalid source configuration: invalid 'watermark_column' type: string. " - + "Valid types are [long, timestamp, timestamptz]"); - read.expand(PBegin.in(testPipeline)); - } - - @Test - public void testFailWhenWatermarkColumnMissingMetrics() { - assumeTrue(useIncrementalScan); - TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - Table table = warehouse.createTable(tableId, TestFixtures.SCHEMA); - table.updateProperties().set("write.metadata.metrics.default", "none").commit(); - IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).withWatermarkColumn("id"); - - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Invalid source configuration: source table"); - thrown.expectMessage( - "not configured to capture lower bound metrics for the specified watermark column 'id'."); - thrown.expectMessage("found 'none'"); - - read.expand(PBegin.in(testPipeline)); - } - - @Test - public void testFailWhenWatermarkTimeUnitUsedWithoutSpecifyingColumn() { - assumeTrue(useIncrementalScan); - TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - warehouse.createTable(tableId, TestFixtures.SCHEMA); - IcebergIO.ReadRows read = - IcebergIO.readRows(catalogConfig()).withCdc().from(tableId).withWatermarkTimeUnit("hours"); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage( - "Invalid source configuration: cannot set 'watermark_time_unit' without " - + "also setting a 'watermark_column' of Long type"); - read.expand(PBegin.in(testPipeline)); - } - @Test public void testSimpleScan() throws Exception { TableIdentifier tableId = @@ -526,164 +456,6 @@ public void testBatchReadBetweenTimestamps() throws IOException { runReadWithBoundary(false, false); } - @Test - public void testWatermarkColumnLongType() throws IOException { - assumeTrue(useIncrementalScan); - TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - Table simpleTable = warehouse.createTable(tableId, TestFixtures.SCHEMA); - - // configure the table to capture full metrics - simpleTable - .updateProperties() - .set(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "id", "full") - .commit(); - - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file1s4.parquet", simpleTable.schema(), TestFixtures.FILE1SNAPSHOT4)) - .commit(); - simpleTable - .newFastAppend() - .appendFile( - warehouse.writeRecords( - "file2s4.parquet", simpleTable.schema(), TestFixtures.FILE2SNAPSHOT4)) - .appendFile( - warehouse.writeRecords( - "file3s4.parquet", simpleTable.schema(), TestFixtures.FILE3SNAPSHOT4)) - .commit(); - - IcebergIO.ReadRows readRows = - IcebergIO.readRows(catalogConfig()) - .from(tableId) - .withCdc() - .streaming(true) - .withStartingStrategy(StartingStrategy.EARLIEST) - .toSnapshot(simpleTable.currentSnapshot().snapshotId()) - .withWatermarkColumn("id") - .withWatermarkTimeUnit("days"); - PCollection output = testPipeline.apply(readRows); - output.apply( - ParDo.of( - new CheckWatermarks( - lowestMillisOf(TestFixtures.FILE1SNAPSHOT4_DATA), - lowestMillisOf(TestFixtures.FILE2SNAPSHOT4_DATA), - lowestMillisOf(TestFixtures.FILE3SNAPSHOT4_DATA), - row -> TimeUnit.DAYS.toMillis(checkStateNotNull(row.getInt64("id")))))); - testPipeline.run().waitUntilFinish(); - } - - private static long lowestMillisOf(List> data) { - long lowestId = data.stream().mapToLong(m -> (long) m.get("id")).min().getAsLong(); - return TimeUnit.DAYS.toMillis(lowestId); - } - - @Test - public void testWatermarkColumnTimestampType() throws IOException { - assumeTrue(useIncrementalScan); - TableIdentifier tableId = TableIdentifier.of("default", testName.getMethodName()); - Schema schema = - Schema.builder() - .addStringField("str") - .addLogicalTypeField("datetime", SqlTypes.DATETIME) - .build(); - org.apache.iceberg.Schema iceSchema = IcebergUtils.beamSchemaToIcebergSchema(schema); - Table simpleTable = warehouse.createTable(tableId, iceSchema); - - LocalDateTime file1Watermark = LocalDateTime.parse("2025-01-01T12:00:00"); - List> data1 = - ImmutableList.of( - ImmutableMap.of("str", "a", "datetime", file1Watermark), - ImmutableMap.of("str", "b", "datetime", file1Watermark.plusMinutes(10)), - ImmutableMap.of("str", "c", "datetime", file1Watermark.plusMinutes(30))); - - LocalDateTime file2Watermark = LocalDateTime.parse("2025-02-01T12:00:00"); - List> data2 = - ImmutableList.of( - ImmutableMap.of("str", "d", "datetime", file2Watermark), - ImmutableMap.of("str", "e", "datetime", file2Watermark.plusMinutes(10)), - ImmutableMap.of("str", "f", "datetime", file2Watermark.plusMinutes(30))); - - LocalDateTime file3Watermark = LocalDateTime.parse("2025-03-01T12:00:00"); - List> data3 = - ImmutableList.of( - ImmutableMap.of("str", "g", "datetime", file3Watermark), - ImmutableMap.of("str", "h", "datetime", file3Watermark.plusMinutes(10)), - ImmutableMap.of("str", "i", "datetime", file3Watermark.plusMinutes(30))); - - // configure the table to capture full metrics - simpleTable - .updateProperties() - .set(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "datetime", "full") - .commit(); - - simpleTable - .newFastAppend() - .appendFile(warehouse.writeData("file1.parquet", iceSchema, data1)) - .commit(); - simpleTable - .newFastAppend() - .appendFile(warehouse.writeData("file2.parquet", iceSchema, data2)) - .appendFile(warehouse.writeData("file3.parquet", iceSchema, data3)) - .commit(); - - IcebergIO.ReadRows readRows = - IcebergIO.readRows(catalogConfig()) - .from(tableId) - .withCdc() - .streaming(true) - .withStartingStrategy(StartingStrategy.EARLIEST) - .toSnapshot(simpleTable.currentSnapshot().snapshotId()) - .withWatermarkColumn("datetime"); - PCollection output = testPipeline.apply(readRows); - output.apply( - ParDo.of( - new CheckWatermarks( - microsToMillis(microsFromTimestamp(file1Watermark)), - microsToMillis(microsFromTimestamp(file2Watermark)), - microsToMillis(microsFromTimestamp(file3Watermark)), - row -> { - LocalDateTime dt = - checkStateNotNull(row.getLogicalTypeValue("datetime", LocalDateTime.class)); - return microsToMillis(microsFromTimestamp(dt)); - }))); - testPipeline.run().waitUntilFinish(); - } - - static class CheckWatermarks extends DoFn { - long file1Watermark; - long file2Watermark; - long file3Watermark; - SerializableFunction getMillisFn; - - CheckWatermarks( - long file1Watermark, - long file2Watermark, - long file3Watermark, - SerializableFunction getMillisFn) { - this.file1Watermark = file1Watermark; - this.file2Watermark = file2Watermark; - this.file3Watermark = file3Watermark; - this.getMillisFn = getMillisFn; - } - - @ProcessElement - public void process(@Element Row row, @Timestamp Instant timestamp) { - Row record = checkStateNotNull(row.getRow(RECORD)); - long expectedMillis = getMillisFn.apply(record); - long actualMillis = timestamp.getMillis(); - - if (expectedMillis >= file3Watermark) { - assertEquals(file3Watermark, actualMillis); - } else if (expectedMillis >= file2Watermark) { - assertEquals(file2Watermark, actualMillis); - } else { - assertEquals(file1Watermark, actualMillis); - } - } - } - public void runWithStartingStrategy(@Nullable StartingStrategy strategy, boolean streaming) throws IOException { assumeTrue(useIncrementalScan); 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 22d505c676f4..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 @@ -104,8 +104,6 @@ public class IcebergSchemaTransformTranslationTest { .withFieldValue("to_timestamp", 456L) .withFieldValue("poll_interval_seconds", 123) .withFieldValue("streaming", true) - .withFieldValue("watermark_column", "id") - .withFieldValue("watermark_time_unit", "nanoseconds") .build(); @Test 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 8c297913d68d..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 @@ -422,7 +422,6 @@ public void testStreamingRead() throws Exception { Map config = new HashMap<>(managedIcebergConfig(tableId())); config.put("streaming", true); config.put("to_snapshot", table.currentSnapshot().snapshotId()); - config.put("watermark_column", "datetime"); PCollection rows = pipeline @@ -472,7 +471,6 @@ public void testReadWriteStreaming() throws IOException { Map readConfig = new HashMap<>(config); readConfig.put("to_timestamp", System.currentTimeMillis()); - readConfig.put("watermark_column", "datetime_tz"); readConfig.put("streaming", true); Map writeConfig = new HashMap<>(config); @@ -749,8 +747,6 @@ public void runReadBetween(boolean useSnapshotBoundary, boolean streaming) throw populateTable(table, "d"); // fourth snapshot Map config = new HashMap<>(managedIcebergConfig(tableId())); - config.put("watermark_column", "nullable_long"); - config.put("watermark_time_unit", "days"); if (useSnapshotBoundary) { config.put("from_snapshot", from.snapshotId()); config.put("to_snapshot", to.snapshotId()); From 30f3ced4d2f841926821db9a2eae68c6eb552c88 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud Date: Wed, 12 Mar 2025 10:39:36 -0400 Subject: [PATCH 32/32] change metric name --- .../org/apache/beam/sdk/io/iceberg/WatchForSnapshots.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 index 8577306f4226..50dc241b39ed 100644 --- 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 @@ -72,8 +72,8 @@ public PCollection>> expand(PBegin input) { private static class SnapshotPollFn extends Watch.Growth.PollFn> { private final Gauge latestSnapshot = Metrics.gauge(SnapshotPollFn.class, "latestSnapshot"); - private final Counter totalSnapshotsObserved = - Metrics.counter(SnapshotPollFn.class, "numSnapshotsObserved"); + private final Counter snapshotsObserved = + Metrics.counter(SnapshotPollFn.class, "snapshotsObserved"); private final IcebergScanConfig scanConfig; private @Nullable Long fromSnapshotId; @@ -121,7 +121,7 @@ private PollResult> getPollResult( "New poll fetched {} snapshots: {}", snapshots.size(), snapshots.stream().map(SnapshotInfo::getSnapshotId).collect(Collectors.toList())); - totalSnapshotsObserved.inc(snapshots.size()); + snapshotsObserved.inc(snapshots.size()); } return isComplete