stevenzwu commented on code in PR #10308: URL: https://github.com/apache/iceberg/pull/10308#discussion_r1624777278
########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java: ########## @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Sink for collecting output during testing. */ +class CollectingSink<T> implements Sink<T> { + private static final long serialVersionUID = 1L; + private static final List<BlockingQueue<Object>> queues = Review Comment: why do we use static list of queues here? why not a non-static queue? are we trying to leverage the static nature for the mini cluster setup (due to serialization and deserialization)? ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java: ########## @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Sink for collecting output during testing. */ +class CollectingSink<T> implements Sink<T> { + private static final long serialVersionUID = 1L; + private static final List<BlockingQueue<Object>> queues = + Collections.synchronizedList(Lists.newArrayListWithExpectedSize(1)); + private static final AtomicInteger numSinks = new AtomicInteger(-1); + private final int index; + + /** Creates a new sink which collects the elements received. */ + CollectingSink() { + this.index = numSinks.incrementAndGet(); + queues.add(new LinkedBlockingQueue<>()); + } + + /** + * Gets all the remaining output received by this {@link Sink}. + * + * @return all the remaining output + */ + List<T> getRemainingOutput() { Review Comment: nit: why is this called `remaining`? also Iceberg style normally doesn't use `get`. ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java: ########## @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +class TestMonitorSource extends OperatorTestBase { + private static final TableChange EMPTY_EVENT = new TableChange(0, 0, 0L, 0L, 0); + private static final RateLimiterStrategy HIGH_RATE = RateLimiterStrategy.perSecond(100.0); + private static final RateLimiterStrategy LOW_RATE = RateLimiterStrategy.perSecond(1.0 / 10000.0); + + @TempDir private File checkpointDir; + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testChangeReaderIterator(boolean withDelete) { + if (withDelete) { + sql.exec( + "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) WITH ('format-version'='2', 'write.upsert.enabled'='true')", + TABLE_NAME); + } else { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + } + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + Table table = tableLoader.loadTable(); + + MonitorSource.SchedulerEventIterator iterator = + new MonitorSource.SchedulerEventIterator(tableLoader, null, Long.MAX_VALUE); + + // For an empty table we get an empty result + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add a single commit and get back the commit data in the event + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + table.refresh(); + long dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + long deleteSize = + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + assertThat(iterator.next()) + .isEqualTo(new TableChange(1, withDelete ? 1 : 0, dataSize, deleteSize, 1)); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add two more commits, but fetch the data in one loop + sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + table.refresh(); + dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + deleteSize = + withDelete Review Comment: nit: extract the code to a small helper method? ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java: ########## @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +class TestMonitorSource extends OperatorTestBase { + private static final TableChange EMPTY_EVENT = new TableChange(0, 0, 0L, 0L, 0); + private static final RateLimiterStrategy HIGH_RATE = RateLimiterStrategy.perSecond(100.0); + private static final RateLimiterStrategy LOW_RATE = RateLimiterStrategy.perSecond(1.0 / 10000.0); + + @TempDir private File checkpointDir; + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testChangeReaderIterator(boolean withDelete) { + if (withDelete) { + sql.exec( + "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) WITH ('format-version'='2', 'write.upsert.enabled'='true')", + TABLE_NAME); + } else { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + } + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + Table table = tableLoader.loadTable(); + + MonitorSource.SchedulerEventIterator iterator = + new MonitorSource.SchedulerEventIterator(tableLoader, null, Long.MAX_VALUE); + + // For an empty table we get an empty result + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add a single commit and get back the commit data in the event + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + table.refresh(); + long dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + long deleteSize = + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + assertThat(iterator.next()) + .isEqualTo(new TableChange(1, withDelete ? 1 : 0, dataSize, deleteSize, 1)); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add two more commits, but fetch the data in one loop + sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + table.refresh(); + dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + deleteSize = + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + table.refresh(); + dataSize += + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + deleteSize += + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + assertThat(iterator.next()) + .isEqualTo(new TableChange(2, withDelete ? 2 : 0, dataSize, deleteSize, 2)); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + } + + /** + * Create a table and check that the source returns the data as new commits arrive to the table. + */ + @Test + void testSource() throws Exception { + sql.exec( + "CREATE TABLE %s (id int, data varchar) " + + "WITH ('flink.max-continuous-empty-commits'='100000')", + TABLE_NAME); + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.enableCheckpointing(1000); + env.setParallelism(1); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + Table table = tableLoader.loadTable(); + DataStream<TableChange> events = + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Creating a stream for inserting data into the table concurrently + ManualSource<RowData> insertSource = + new ManualSource<>(env, InternalTypeInfo.of(FlinkSchemaUtil.convert(table.schema()))); + FlinkSink.forRowData(insertSource.getDataStream()) + .tableLoader(tableLoader) + .uidPrefix("iceberg-sink") + .append(); + + // Sink to collect the results + CollectingSink<TableChange> result = new CollectingSink<>(); + events.sinkTo(result); + + // First result is an empty event + env.executeAsync("Table Change Source Test"); + assertThat(result.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); + + // Insert some data + insertSource.sendRecord(GenericRowData.of(1, StringData.fromString("a"))); + // Wait until the changes are committed + Awaitility.await() + .until( + () -> { + table.refresh(); + return table.currentSnapshot() != null; + }); + + table.refresh(); + long size = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + + // Wait until the first non-empty event has arrived, and check the expected result + Awaitility.await() + .until( + () -> { + TableChange newEvent = result.poll(Duration.ofSeconds(5L)); + // Fetch every empty event from the beginning + while (newEvent.equals(EMPTY_EVENT)) { + newEvent = result.poll(Duration.ofSeconds(5L)); + } + + // The first non-empty event should contain the expected value + return newEvent.equals(new TableChange(1, 0, size, 0L, 1)); + }); + } + + /** Check that the {@link MonitorSource} operator state is restored correctly. */ + @Test + void testStateRestore(@TempDir File savepointDir) throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.enableCheckpointing(1000); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + DataStream<TableChange> events = + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Sink to collect the results + CollectingSink<TableChange> result = new CollectingSink<>(); + events.sinkTo(result); + + // Start the job + Configuration conf; + JobClient jobClient = null; + AtomicReference<TableChange> firstNonEmptyEvent = new AtomicReference<>(); + try { + jobClient = env.executeAsync("Table Change Source Test"); + + Awaitility.await() + .until( + () -> { + TableChange newEvent = result.poll(Duration.ofSeconds(5L)); + // Fetch every empty event from the beginning + while (newEvent.equals(EMPTY_EVENT)) { + newEvent = result.poll(Duration.ofSeconds(5L)); + } + + // The first non-empty event should contain the expected value + firstNonEmptyEvent.set(newEvent); + return true; + }); + } finally { + // Stop with savepoint + conf = closeJobClient(jobClient, savepointDir); + } + + // Restore from savepoint, create the same topology with a different env + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + events = + env.fromSource( + new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + CollectingSink<TableChange> resultWithSavepoint = new CollectingSink<>(); + events.sinkTo(resultWithSavepoint); + + // Make sure that the job with restored source does not read new records from the table + JobClient clientWithSavepoint = null; + try { + clientWithSavepoint = env.executeAsync("Table Change Source test with savepoint"); + + assertThat(resultWithSavepoint.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); + } finally { + closeJobClient(clientWithSavepoint, null); + } + + // Restore without savepoint + env = StreamExecutionEnvironment.getExecutionEnvironment(); + events = + env.fromSource( + new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + CollectingSink<TableChange> resultWithoutSavepoint = new CollectingSink<>(); + events.sinkTo(resultWithoutSavepoint); + + // Make sure that a new job without state reads the event as expected + JobClient clientWithoutSavepoint = null; + try { + clientWithoutSavepoint = env.executeAsync("Table Change Source Test without savepoint"); + assertThat(resultWithoutSavepoint.poll(Duration.ofSeconds(5L))) + .isEqualTo(firstNonEmptyEvent.get()); + } finally { + closeJobClient(clientWithoutSavepoint); + } + } + + @Test + void testNotOneParallelismThrows() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .setParallelism(2) + .print(); + + assertThatThrownBy(env::execute) + .isInstanceOf(JobExecutionException.class) + .cause() + .cause() + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Parallelism should be set to 1"); + } + + @Test + void testMaxReadBack() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + MonitorSource.SchedulerEventIterator iterator = + new MonitorSource.SchedulerEventIterator(tableLoader, null, 1); + + // For a single maxReadBack we only get a single change + assertThat(iterator.next().commitNum()).isEqualTo(1); + + iterator = new MonitorSource.SchedulerEventIterator(tableLoader, null, 2); + + // For maxReadBack we 2 we get 2 changes Review Comment: typo: `Expecting 2 commits/snapshots for maxReadBack =2` ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.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.iceberg.flink.maintenance.operator; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +/** + * Junit 5 extension for running Flink SQL queries. {@link + * org.apache.flink.test.junit5.MiniClusterExtension} is used for executing the SQL batch jobs. + */ +public class FlinkSqlExtension implements BeforeEachCallback, AfterEachCallback { + private final String catalogName; + private final Map<String, String> catalogProperties; + private final String databaseName; + private final Path warehouse; + private final CatalogLoader catalogLoader; + private TableEnvironment tableEnvironment; + + public FlinkSqlExtension( + String catalogName, Map<String, String> catalogProperties, String databaseName) { + this.catalogName = catalogName; + this.catalogProperties = Maps.newHashMap(catalogProperties); + this.databaseName = databaseName; + + // Add temporary dir as a warehouse location + try { + this.warehouse = Files.createTempDirectory("warehouse"); Review Comment: do we need to pass in a `@TempDir` that can auto clean up files/dirs? ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java: ########## @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +class TestMonitorSource extends OperatorTestBase { + private static final TableChange EMPTY_EVENT = new TableChange(0, 0, 0L, 0L, 0); + private static final RateLimiterStrategy HIGH_RATE = RateLimiterStrategy.perSecond(100.0); + private static final RateLimiterStrategy LOW_RATE = RateLimiterStrategy.perSecond(1.0 / 10000.0); + + @TempDir private File checkpointDir; + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testChangeReaderIterator(boolean withDelete) { + if (withDelete) { + sql.exec( + "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) WITH ('format-version'='2', 'write.upsert.enabled'='true')", + TABLE_NAME); + } else { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); Review Comment: do you intend v1 table here? if yes, maybe explicitly set the `format-version` as the default may change ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.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.iceberg.flink.maintenance.operator; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +/** + * Junit 5 extension for running Flink SQL queries. {@link + * org.apache.flink.test.junit5.MiniClusterExtension} is used for executing the SQL batch jobs. + */ +public class FlinkSqlExtension implements BeforeEachCallback, AfterEachCallback { + private final String catalogName; + private final Map<String, String> catalogProperties; + private final String databaseName; + private final Path warehouse; + private final CatalogLoader catalogLoader; + private TableEnvironment tableEnvironment; + + public FlinkSqlExtension( + String catalogName, Map<String, String> catalogProperties, String databaseName) { + this.catalogName = catalogName; + this.catalogProperties = Maps.newHashMap(catalogProperties); + this.databaseName = databaseName; + + // Add temporary dir as a warehouse location + try { + this.warehouse = Files.createTempDirectory("warehouse"); + } catch (IOException e) { + throw new RuntimeException(e); + } + this.catalogProperties.put( + CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouse)); + this.catalogLoader = + CatalogLoader.hadoop(catalogName, new Configuration(), this.catalogProperties); + } + + @Override + public void beforeEach(ExtensionContext context) { + // We need to recreate the tableEnvironment for every test as the minicluster is recreated + this.tableEnvironment = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + exec("CREATE CATALOG %s WITH %s", catalogName, toWithClause(catalogProperties)); + exec("CREATE DATABASE %s.%s", catalogName, databaseName); + exec("USE CATALOG %s", catalogName); + exec("USE %s", databaseName); + } + + @Override + public void afterEach(ExtensionContext context) throws IOException { + List<Row> tables = exec("SHOW TABLES"); + tables.forEach(t -> exec("DROP TABLE IF EXISTS %s", t.getField(0))); + exec("USE CATALOG default_catalog"); + exec("DROP CATALOG IF EXISTS %s", catalogName); + Files.walk(warehouse).sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } + + /** + * Executes an SQL query with the given parameters. The parameter substitution is done by {@link + * String#format(String, Object...)}. + * + * @param query to run + * @param parameters to substitute to the query + * @return The {@link Row}s returned by the query + */ + public List<Row> exec(String query, Object... parameters) { + TableResult tableResult = tableEnvironment.executeSql(String.format(query, parameters)); + try (CloseableIterator<Row> iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + /** + * Returns the {@link TableLoader} which could be used to access the given table. + * + * @param name of the table + * @return the {@link TableLoader} for the table + */ + public TableLoader tableLoader(String name) { Review Comment: nit: name -> tableName ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java: ########## @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +class TestMonitorSource extends OperatorTestBase { + private static final TableChange EMPTY_EVENT = new TableChange(0, 0, 0L, 0L, 0); + private static final RateLimiterStrategy HIGH_RATE = RateLimiterStrategy.perSecond(100.0); + private static final RateLimiterStrategy LOW_RATE = RateLimiterStrategy.perSecond(1.0 / 10000.0); + + @TempDir private File checkpointDir; + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testChangeReaderIterator(boolean withDelete) { + if (withDelete) { + sql.exec( + "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) WITH ('format-version'='2', 'write.upsert.enabled'='true')", + TABLE_NAME); + } else { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + } + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + Table table = tableLoader.loadTable(); + + MonitorSource.SchedulerEventIterator iterator = + new MonitorSource.SchedulerEventIterator(tableLoader, null, Long.MAX_VALUE); + + // For an empty table we get an empty result + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add a single commit and get back the commit data in the event + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + table.refresh(); + long dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + long deleteSize = + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + assertThat(iterator.next()) + .isEqualTo(new TableChange(1, withDelete ? 1 : 0, dataSize, deleteSize, 1)); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add two more commits, but fetch the data in one loop + sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + table.refresh(); + dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + deleteSize = + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + table.refresh(); + dataSize += + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + deleteSize += + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + assertThat(iterator.next()) + .isEqualTo(new TableChange(2, withDelete ? 2 : 0, dataSize, deleteSize, 2)); Review Comment: nit: tied to some previous comment. it is a bit hard to read/remember what each arg is for (and they are all ints or longs). ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java: ########## @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +class TestMonitorSource extends OperatorTestBase { + private static final TableChange EMPTY_EVENT = new TableChange(0, 0, 0L, 0L, 0); + private static final RateLimiterStrategy HIGH_RATE = RateLimiterStrategy.perSecond(100.0); + private static final RateLimiterStrategy LOW_RATE = RateLimiterStrategy.perSecond(1.0 / 10000.0); + + @TempDir private File checkpointDir; + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testChangeReaderIterator(boolean withDelete) { + if (withDelete) { + sql.exec( + "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) WITH ('format-version'='2', 'write.upsert.enabled'='true')", + TABLE_NAME); + } else { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + } + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + Table table = tableLoader.loadTable(); + + MonitorSource.SchedulerEventIterator iterator = + new MonitorSource.SchedulerEventIterator(tableLoader, null, Long.MAX_VALUE); + + // For an empty table we get an empty result + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add a single commit and get back the commit data in the event + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + table.refresh(); + long dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + long deleteSize = + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + assertThat(iterator.next()) + .isEqualTo(new TableChange(1, withDelete ? 1 : 0, dataSize, deleteSize, 1)); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add two more commits, but fetch the data in one loop + sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + table.refresh(); + dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + deleteSize = + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + table.refresh(); + dataSize += + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + deleteSize += + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + assertThat(iterator.next()) + .isEqualTo(new TableChange(2, withDelete ? 2 : 0, dataSize, deleteSize, 2)); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + } + + /** + * Create a table and check that the source returns the data as new commits arrive to the table. + */ + @Test + void testSource() throws Exception { + sql.exec( + "CREATE TABLE %s (id int, data varchar) " + + "WITH ('flink.max-continuous-empty-commits'='100000')", + TABLE_NAME); + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.enableCheckpointing(1000); + env.setParallelism(1); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + Table table = tableLoader.loadTable(); + DataStream<TableChange> events = + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Creating a stream for inserting data into the table concurrently + ManualSource<RowData> insertSource = Review Comment: wondering if we need the `ManualSource`. can we use the appender? e.g. ``` GenericAppenderHelper dataAppender = new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); List<Record> batch1 = RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ``` ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java: ########## @@ -0,0 +1,344 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.time.Duration; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +class TestMonitorSource extends OperatorTestBase { + private static final TableChange EMPTY_EVENT = new TableChange(0, 0, 0L, 0L, 0); + private static final RateLimiterStrategy HIGH_RATE = RateLimiterStrategy.perSecond(100.0); + private static final RateLimiterStrategy LOW_RATE = RateLimiterStrategy.perSecond(1.0 / 10000.0); + + @TempDir private File checkpointDir; + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testChangeReaderIterator(boolean withDelete) { + if (withDelete) { + sql.exec( + "CREATE TABLE %s (id int, data varchar, PRIMARY KEY(`id`) NOT ENFORCED) WITH ('format-version'='2', 'write.upsert.enabled'='true')", + TABLE_NAME); + } else { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + } + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + Table table = tableLoader.loadTable(); + + MonitorSource.SchedulerEventIterator iterator = + new MonitorSource.SchedulerEventIterator(tableLoader, null, Long.MAX_VALUE); + + // For an empty table we get an empty result + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add a single commit and get back the commit data in the event + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + table.refresh(); + long dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + long deleteSize = + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + assertThat(iterator.next()) + .isEqualTo(new TableChange(1, withDelete ? 1 : 0, dataSize, deleteSize, 1)); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + + // Add two more commits, but fetch the data in one loop + sql.exec("INSERT INTO %s VALUES (2, 'b')", TABLE_NAME); + table.refresh(); + dataSize = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + deleteSize = + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + table.refresh(); + dataSize += + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + deleteSize += + withDelete + ? table + .currentSnapshot() + .addedDeleteFiles(table.io()) + .iterator() + .next() + .fileSizeInBytes() + : 0; + assertThat(iterator.next()) + .isEqualTo(new TableChange(2, withDelete ? 2 : 0, dataSize, deleteSize, 2)); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + } + + /** + * Create a table and check that the source returns the data as new commits arrive to the table. + */ + @Test + void testSource() throws Exception { + sql.exec( + "CREATE TABLE %s (id int, data varchar) " + + "WITH ('flink.max-continuous-empty-commits'='100000')", + TABLE_NAME); + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.enableCheckpointing(1000); + env.setParallelism(1); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + Table table = tableLoader.loadTable(); + DataStream<TableChange> events = + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Creating a stream for inserting data into the table concurrently + ManualSource<RowData> insertSource = + new ManualSource<>(env, InternalTypeInfo.of(FlinkSchemaUtil.convert(table.schema()))); + FlinkSink.forRowData(insertSource.getDataStream()) + .tableLoader(tableLoader) + .uidPrefix("iceberg-sink") + .append(); + + // Sink to collect the results + CollectingSink<TableChange> result = new CollectingSink<>(); + events.sinkTo(result); + + // First result is an empty event + env.executeAsync("Table Change Source Test"); + assertThat(result.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); + + // Insert some data + insertSource.sendRecord(GenericRowData.of(1, StringData.fromString("a"))); + // Wait until the changes are committed + Awaitility.await() + .until( + () -> { + table.refresh(); + return table.currentSnapshot() != null; + }); + + table.refresh(); + long size = + table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + + // Wait until the first non-empty event has arrived, and check the expected result + Awaitility.await() + .until( + () -> { + TableChange newEvent = result.poll(Duration.ofSeconds(5L)); + // Fetch every empty event from the beginning + while (newEvent.equals(EMPTY_EVENT)) { + newEvent = result.poll(Duration.ofSeconds(5L)); + } + + // The first non-empty event should contain the expected value + return newEvent.equals(new TableChange(1, 0, size, 0L, 1)); + }); + } + + /** Check that the {@link MonitorSource} operator state is restored correctly. */ + @Test + void testStateRestore(@TempDir File savepointDir) throws Exception { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.enableCheckpointing(1000); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + DataStream<TableChange> events = + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Sink to collect the results + CollectingSink<TableChange> result = new CollectingSink<>(); + events.sinkTo(result); + + // Start the job + Configuration conf; + JobClient jobClient = null; + AtomicReference<TableChange> firstNonEmptyEvent = new AtomicReference<>(); + try { + jobClient = env.executeAsync("Table Change Source Test"); + + Awaitility.await() + .until( + () -> { + TableChange newEvent = result.poll(Duration.ofSeconds(5L)); + // Fetch every empty event from the beginning + while (newEvent.equals(EMPTY_EVENT)) { + newEvent = result.poll(Duration.ofSeconds(5L)); + } + + // The first non-empty event should contain the expected value + firstNonEmptyEvent.set(newEvent); + return true; + }); + } finally { + // Stop with savepoint + conf = closeJobClient(jobClient, savepointDir); + } + + // Restore from savepoint, create the same topology with a different env + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + events = + env.fromSource( + new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + CollectingSink<TableChange> resultWithSavepoint = new CollectingSink<>(); + events.sinkTo(resultWithSavepoint); + + // Make sure that the job with restored source does not read new records from the table + JobClient clientWithSavepoint = null; + try { + clientWithSavepoint = env.executeAsync("Table Change Source test with savepoint"); + + assertThat(resultWithSavepoint.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); + } finally { + closeJobClient(clientWithSavepoint, null); + } + + // Restore without savepoint + env = StreamExecutionEnvironment.getExecutionEnvironment(); + events = + env.fromSource( + new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + CollectingSink<TableChange> resultWithoutSavepoint = new CollectingSink<>(); + events.sinkTo(resultWithoutSavepoint); + + // Make sure that a new job without state reads the event as expected + JobClient clientWithoutSavepoint = null; + try { + clientWithoutSavepoint = env.executeAsync("Table Change Source Test without savepoint"); + assertThat(resultWithoutSavepoint.poll(Duration.ofSeconds(5L))) + .isEqualTo(firstNonEmptyEvent.get()); + } finally { + closeJobClient(clientWithoutSavepoint); + } + } + + @Test + void testNotOneParallelismThrows() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .setParallelism(2) + .print(); + + assertThatThrownBy(env::execute) + .isInstanceOf(JobExecutionException.class) + .cause() Review Comment: instead of 3 cause calls, does `rootCause` work? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
