stevenzwu commented on code in PR #8553: URL: https://github.com/apache/iceberg/pull/8553#discussion_r1401000047
########## flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java: ########## @@ -0,0 +1,98 @@ +/* + * 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.source.reader; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; + +/** + * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics + * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link + * WatermarkExtractorRecordEmitter} along with the actual records. + */ +@Internal +public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { + private final int eventTimeFieldId; + private final String eventTimeFieldName; + private final TimeUnit timeUnit; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param eventTimeFieldName The column which should be used as an event time + * @param timeUnit Used for converting the long value to epoch milliseconds + */ + public ColumnStatsWatermarkExtractor( + Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { + Types.NestedField field = schema.findField(eventTimeFieldName); + TypeID typeID = field.type().typeId(); + Preconditions.checkArgument( + typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), + "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", + typeID); + this.eventTimeFieldId = field.fieldId(); + this.eventTimeFieldName = eventTimeFieldName; + // Use the timeUnit only for Long columns. + this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; + } + + @VisibleForTesting + ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { + this.eventTimeFieldId = eventTimeFieldId; + this.eventTimeFieldName = eventTimeFieldName; + this.timeUnit = TimeUnit.MICROSECONDS; + } + + /** + * Get the watermark for a split using column statistics. + * + * @param split The split + * @return The watermark + * @throws IllegalArgumentException if there is no statistics for the column + */ + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> { + Preconditions.checkArgument( + scanTask.file().lowerBounds() != null + && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, + "Missing statistics for columnName = %s, columnId = %s in file = %s", Review Comment: nit: Iceberg style doesn't use variable name style in error msg, like `columnName`. I also feel we don't need column id in the error msg. maybe `Missing statistics for column %s in file %s` ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java: ########## @@ -0,0 +1,434 @@ +/* + * 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.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + * <ul> + * <li>- Ordering of the splits + * <li>- Emitting of watermarks + * <li>- Firing windows based on watermarks + * </ul> + * + * <p>The test generates 4 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * <li>- Split 4 - Watermark 6 min + * </ul> + * + * <p>Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + * <p>Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + * <ul> + * <li>- Split 2, 3 + * <li>- Split 4 + * <li>- Split 1 + * </ul> + * + * <p>As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + * <p>Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List<Record> expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.<RowData>noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + DataStream<RowData> windowed = + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction<RowData, RowData, TimeWindow>() { + @Override + public void apply( + TimeWindow window, Iterable<RowData> values, Collector<RowData> out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicLong count = new AtomicLong(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + } + }); + + try (CloseableIterator<RowData> resultIterator = windowed.collectAsync()) { + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Assert.assertEquals( + ImmutableSet.of(row(0, RECORD_NUM_FOR_2_SPLITS), row(300000, 2)), + waitForRecords(resultIterator, 2)); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record"))), + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Assert.assertEquals(ImmutableSet.of(row(6000000, 3)), waitForRecords(resultIterator, 1)); + } + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + * <ul> + * <li>- Emitting of watermarks + * <li>- Watermark alignment + * </ul> + * + * <p>The test generates 3 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * </ul> + * + * The splits are read in the following order: + * + * <ul> + * <li>- Split 2, 3 (Task Manager 1, Task Manager 2) + * <li>- Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + * </ul> + * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + * <p>The status of the watermark alignment is checked by the alignment related metrics. + * + * <p>Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + * <p>After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) Review Comment: why not move these 3 configs also inside `sourceBuilder()`? seem common ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java: ########## @@ -0,0 +1,434 @@ +/* + * 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.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { Review Comment: how long does this unit test class run? ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java: ########## @@ -0,0 +1,434 @@ +/* + * 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.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + * <ul> + * <li>- Ordering of the splits + * <li>- Emitting of watermarks + * <li>- Firing windows based on watermarks + * </ul> + * + * <p>The test generates 4 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * <li>- Split 4 - Watermark 6 min + * </ul> + * + * <p>Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + * <p>Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + * <ul> + * <li>- Split 2, 3 + * <li>- Split 4 + * <li>- Split 1 + * </ul> + * + * <p>As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + * <p>Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List<Record> expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.<RowData>noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + DataStream<RowData> windowed = + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction<RowData, RowData, TimeWindow>() { + @Override + public void apply( + TimeWindow window, Iterable<RowData> values, Collector<RowData> out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicLong count = new AtomicLong(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + } + }); + + try (CloseableIterator<RowData> resultIterator = windowed.collectAsync()) { + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Assert.assertEquals( + ImmutableSet.of(row(0, RECORD_NUM_FOR_2_SPLITS), row(300000, 2)), + waitForRecords(resultIterator, 2)); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record"))), + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Assert.assertEquals(ImmutableSet.of(row(6000000, 3)), waitForRecords(resultIterator, 1)); + } + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + * <ul> + * <li>- Emitting of watermarks + * <li>- Watermark alignment + * </ul> + * + * <p>The test generates 3 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * </ul> + * + * The splits are read in the following order: + * + * <ul> + * <li>- Split 2, 3 (Task Manager 1, Task Manager 2) + * <li>- Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + * </ul> + * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + * <p>The status of the watermark alignment is checked by the alignment related metrics. + * + * <p>Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + * <p>After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.<RowData>noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + try (CloseableIterator<RowData> resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); + + // Check that the read the non-blocked data + // The first RECORD_NUM_FOR_2_SPLITS should be read + // 1 or more from the runaway reader should be arrived depending on thread scheduling + waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .atMost(120, TimeUnit.SECONDS) + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge<Long> drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + List<Record> newBatch1 = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + List<Record> newBatch2 = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + dataAppender.appendToTable( + dataAppender.writeFile(newBatch1), dataAppender.writeFile(newBatch2)); + // The records received will highly depend on scheduling + // We minimally get 3 records from the non-blocked reader + // We might get 1 record from the blocked reader (as part of the previous batch - + // file_1-recordTs_100) + // We might get 3 records form the non-blocked reader if it gets both new splits + waitForRecords(resultIterator, 3); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await() Review Comment: should we reduce the poll interval from default 100 ms to sth smaller like 2 ms? ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.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.iceberg.flink.source; + +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; + +public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { + // Increment ts by 15 minutes for each generateRecords batch + private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); + // Within a batch, increment ts by 1 second + private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); + + private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + + @Override + protected IcebergSource.Builder<RowData> sourceBuilder() { + return IcebergSource.<RowData>builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA); + } + + @Override + protected Schema schema() { + return TestFixtures.TS_SCHEMA; + } + + @Override + protected List<Record> generateRecords(int numRecords, long seed) { + // Override the ts field to create a more realistic situation for event time alignment + tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); + return RandomGenericData.generate(schema(), numRecords, seed).stream() + .peek( + record -> { + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), + ZoneId.of("Z")); + record.setField("ts", ts); + }) + .collect(Collectors.toList()); + } + + /** + * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves + * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates + * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the + * {@link LocalDateTime} to a Long type so that Comparators can continue to work. + */ + @Override + protected void assertRecords(Table table, List<Record> expectedRecords, Duration timeout) + throws Exception { + List<Record> expectedNormalized = convertTimestampField(expectedRecords); + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> { + SimpleDataUtil.equalsRecords( + expectedNormalized, + convertTimestampField(SimpleDataUtil.tableRecords(table)), + table.schema()); + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertTimestampField(SimpleDataUtil.tableRecords(table)), + table.schema()); + }); + } + + private List<Record> convertTimestampField(List<Record> records) { Review Comment: nit: maybe `convertLocalDateTimeToMilli` to be more explicit? ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java: ########## @@ -0,0 +1,434 @@ +/* + * 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.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + * <ul> + * <li>- Ordering of the splits + * <li>- Emitting of watermarks + * <li>- Firing windows based on watermarks + * </ul> + * + * <p>The test generates 4 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * <li>- Split 4 - Watermark 6 min + * </ul> + * + * <p>Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + * <p>Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + * <ul> + * <li>- Split 2, 3 + * <li>- Split 4 + * <li>- Split 1 + * </ul> + * + * <p>As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + * <p>Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List<Record> expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.<RowData>noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + DataStream<RowData> windowed = + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction<RowData, RowData, TimeWindow>() { + @Override + public void apply( + TimeWindow window, Iterable<RowData> values, Collector<RowData> out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicLong count = new AtomicLong(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + } + }); + + try (CloseableIterator<RowData> resultIterator = windowed.collectAsync()) { + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Assert.assertEquals( + ImmutableSet.of(row(0, RECORD_NUM_FOR_2_SPLITS), row(300000, 2)), Review Comment: nit: change the large mills values to TimeUnit.toMillis for readability ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java: ########## @@ -0,0 +1,434 @@ +/* + * 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.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + * <ul> + * <li>- Ordering of the splits + * <li>- Emitting of watermarks + * <li>- Firing windows based on watermarks + * </ul> + * + * <p>The test generates 4 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * <li>- Split 4 - Watermark 6 min + * </ul> + * + * <p>Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + * <p>Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + * <ul> + * <li>- Split 2, 3 + * <li>- Split 4 + * <li>- Split 1 + * </ul> + * + * <p>As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + * <p>Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List<Record> expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.<RowData>noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + DataStream<RowData> windowed = + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction<RowData, RowData, TimeWindow>() { + @Override + public void apply( + TimeWindow window, Iterable<RowData> values, Collector<RowData> out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicLong count = new AtomicLong(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + } + }); + + try (CloseableIterator<RowData> resultIterator = windowed.collectAsync()) { + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Assert.assertEquals( + ImmutableSet.of(row(0, RECORD_NUM_FOR_2_SPLITS), row(300000, 2)), + waitForRecords(resultIterator, 2)); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record"))), + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Assert.assertEquals(ImmutableSet.of(row(6000000, 3)), waitForRecords(resultIterator, 1)); + } + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + * <ul> + * <li>- Emitting of watermarks + * <li>- Watermark alignment + * </ul> + * + * <p>The test generates 3 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * </ul> + * + * The splits are read in the following order: + * + * <ul> + * <li>- Split 2, 3 (Task Manager 1, Task Manager 2) + * <li>- Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + * </ul> + * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + * <p>The status of the watermark alignment is checked by the alignment related metrics. + * + * <p>Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + * <p>After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) Review Comment: nit: even smaller, like 2 ms? ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java: ########## @@ -0,0 +1,434 @@ +/* + * 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.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + * <ul> + * <li>- Ordering of the splits + * <li>- Emitting of watermarks + * <li>- Firing windows based on watermarks + * </ul> + * + * <p>The test generates 4 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * <li>- Split 4 - Watermark 6 min + * </ul> + * + * <p>Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + * <p>Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + * <ul> + * <li>- Split 2, 3 + * <li>- Split 4 + * <li>- Split 1 + * </ul> + * + * <p>As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + * <p>Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List<Record> expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.<RowData>noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + DataStream<RowData> windowed = + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction<RowData, RowData, TimeWindow>() { + @Override + public void apply( + TimeWindow window, Iterable<RowData> values, Collector<RowData> out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicLong count = new AtomicLong(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + } + }); + + try (CloseableIterator<RowData> resultIterator = windowed.collectAsync()) { + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Assert.assertEquals( + ImmutableSet.of(row(0, RECORD_NUM_FOR_2_SPLITS), row(300000, 2)), + waitForRecords(resultIterator, 2)); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record"))), + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Assert.assertEquals(ImmutableSet.of(row(6000000, 3)), waitForRecords(resultIterator, 1)); + } + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + * <ul> + * <li>- Emitting of watermarks + * <li>- Watermark alignment + * </ul> + * + * <p>The test generates 3 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * </ul> + * + * The splits are read in the following order: + * + * <ul> + * <li>- Split 2, 3 (Task Manager 1, Task Manager 2) + * <li>- Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + * </ul> + * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + * <p>The status of the watermark alignment is checked by the alignment related metrics. + * + * <p>Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + * <p>After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.<RowData>noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + try (CloseableIterator<RowData> resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); + + // Check that the read the non-blocked data + // The first RECORD_NUM_FOR_2_SPLITS should be read + // 1 or more from the runaway reader should be arrived depending on thread scheduling + waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .atMost(120, TimeUnit.SECONDS) + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge<Long> drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + List<Record> newBatch1 = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + List<Record> newBatch2 = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + dataAppender.appendToTable( + dataAppender.writeFile(newBatch1), dataAppender.writeFile(newBatch2)); + // The records received will highly depend on scheduling + // We minimally get 3 records from the non-blocked reader + // We might get 1 record from the blocked reader (as part of the previous batch - + // file_1-recordTs_100) + // We might get 3 records form the non-blocked reader if it gets both new splits + waitForRecords(resultIterator, 3); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await() + .atMost(120, TimeUnit.SECONDS) Review Comment: is 120s necessary? can we reduce it sth a lot smaller like 30s or 10s? ########## flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java: ########## @@ -0,0 +1,434 @@ +/* + * 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.source; + +import static org.apache.flink.connector.testframe.utils.ConnectorTestConstants.DEFAULT_COLLECT_DATA_TIMEOUT; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(new Configuration())) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + * <ul> + * <li>- Ordering of the splits + * <li>- Emitting of watermarks + * <li>- Firing windows based on watermarks + * </ul> + * + * <p>The test generates 4 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * <li>- Split 4 - Watermark 6 min + * </ul> + * + * <p>Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + * <p>Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + * <ul> + * <li>- Split 2, 3 + * <li>- Split 4 + * <li>- Split 1 + * </ul> + * + * <p>As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + * <p>Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List<Record> expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.<RowData>noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + DataStream<RowData> windowed = + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction<RowData, RowData, TimeWindow>() { + @Override + public void apply( + TimeWindow window, Iterable<RowData> values, Collector<RowData> out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicLong count = new AtomicLong(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + } + }); + + try (CloseableIterator<RowData> resultIterator = windowed.collectAsync()) { + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Assert.assertEquals( + ImmutableSet.of(row(0, RECORD_NUM_FOR_2_SPLITS), row(300000, 2)), + waitForRecords(resultIterator, 2)); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record"))), + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Assert.assertEquals(ImmutableSet.of(row(6000000, 3)), waitForRecords(resultIterator, 1)); + } + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + * <ul> + * <li>- Emitting of watermarks + * <li>- Watermark alignment + * </ul> + * + * <p>The test generates 3 splits + * + * <ul> + * <li>- Split 1 - Watermark 100 min + * <li>- Split 2, 3 - Watermark 0 min + * </ul> + * + * The splits are read in the following order: + * + * <ul> + * <li>- Split 2, 3 (Task Manager 1, Task Manager 2) + * <li>- Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + * </ul> + * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + * <p>The status of the watermark alignment is checked by the alignment related metrics. + * + * <p>Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + * <p>After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List<Record> batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream<RowData> stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.<RowData>noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + try (CloseableIterator<RowData> resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); + + // Check that the read the non-blocked data + // The first RECORD_NUM_FOR_2_SPLITS should be read + // 1 or more from the runaway reader should be arrived depending on thread scheduling + waitForRecords(resultIterator, RECORD_NUM_FOR_2_SPLITS + 1); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .atMost(120, TimeUnit.SECONDS) + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge<Long> drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + List<Record> newBatch1 = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + List<Record> newBatch2 = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + dataAppender.appendToTable( + dataAppender.writeFile(newBatch1), dataAppender.writeFile(newBatch2)); + // The records received will highly depend on scheduling + // We minimally get 3 records from the non-blocked reader + // We might get 1 record from the blocked reader (as part of the previous batch - + // file_1-recordTs_100) + // We might get 3 records form the non-blocked reader if it gets both new splits + waitForRecords(resultIterator, 3); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await() + .atMost(120, TimeUnit.SECONDS) + .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); + + // Add some new records which should unblock the throttled reader + batch = + ImmutableList.of( + generateRecord(110, "file_5-recordTs_110"), Review Comment: nit: maybe the watermark can be 90 and 91? -- 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