pvary commented on code in PR #10308:
URL: https://github.com/apache/iceberg/pull/10308#discussion_r1625128576


##########
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:
   It will be used for testing the different MaintenanceTasks as well.
   I wouldn't added the `ManualSource` for this use-case, but since it was 
added, I think reusing it here makes it easier to understand the real purpose 
of the test if we don't have to mix other helpers to the process.



-- 
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: issues-unsubscr...@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to