stevenzwu commented on code in PR #9308: URL: https://github.com/apache/iceberg/pull/9308#discussion_r1431861322
########## flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java: ########## @@ -46,6 +51,8 @@ class IcebergSourceSplitReader<T> implements SplitReader<RecordAndPosition<T>, I private final SerializableComparator<IcebergSourceSplit> splitComparator; private final int indexOfSubtask; private final Queue<IcebergSourceSplit> splits; + private final Set<String> pausedSplits; Review Comment: add a comment to explain that synchronization/lock is not needed because all accesses are from single reader thread from `SingleThreadMultiplexSourceReaderBase` ########## flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceSplitReader.java: ########## @@ -0,0 +1,167 @@ +/* + * 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 static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; +import org.apache.iceberg.types.Types; +import org.awaitility.Awaitility; +import org.awaitility.core.ConditionTimeoutException; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceSplitReader { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private static final Schema SCHEMA = + new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + @Test + public void testPause() throws Exception { + IcebergSourceSplitReader<RowData> reader = reader(); + + List<List<Record>> records = + ImmutableList.of( + RandomGenericData.generate(SCHEMA, 2, 0L), RandomGenericData.generate(SCHEMA, 3, 1L)); + + IcebergSourceSplit split = + ReaderUtil.createSplit(records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY); + + // Add the new split to the reader + reader.handleSplitsChanges(new SplitsAddition<>(ImmutableList.of(split))); + + // Fetch the first batch, and check the result size + ArrayBatchRecords<RecordAndPosition<RowData>> result = (ArrayBatchRecords) reader.fetch(); + assertThat(result.numberOfRecords()).isEqualTo(2); + + // Pause the reading of the split, and check that the read is blocked when trying to fetch new + // records + reader.pauseOrResumeSplits(ImmutableList.of(split), ImmutableList.of()); + assertThatThrownBy( + () -> + Awaitility.await().atMost(2, TimeUnit.SECONDS).until(() -> reader.fetch() != null)) + .isInstanceOf(ConditionTimeoutException.class) + .hasMessageContaining("was not fulfilled within"); + + // Unpause the reading of the split, and check the result size + reader.pauseOrResumeSplits(ImmutableList.of(), ImmutableList.of(split)); + result = (ArrayBatchRecords) reader.fetch(); + assertThat(result.numberOfRecords()).isEqualTo(3); + } + + @Test + public void testWakeup() throws Exception { + IcebergSourceSplitReader<RowData> reader = reader(); + + List<List<Record>> records = + ImmutableList.of( + RandomGenericData.generate(SCHEMA, 2, 0L), RandomGenericData.generate(SCHEMA, 3, 1L)); + IcebergSourceSplit split = + ReaderUtil.createSplit(records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY); + + // Add the new split to the reader + reader.handleSplitsChanges(new SplitsAddition<>(ImmutableList.of(split))); + + // Fetch the first batch, and check the result size + ArrayBatchRecords<RecordAndPosition<RowData>> result = (ArrayBatchRecords) reader.fetch(); + assertThat(result.numberOfRecords()).isEqualTo(2); + + // Pause the reading of the split, and check that the read is blocked when trying to fetch new + // records + reader.pauseOrResumeSplits(ImmutableList.of(split), ImmutableList.of()); + + assertThatThrownBy( + () -> + Awaitility.await().atMost(2, TimeUnit.SECONDS).until(() -> reader.fetch() != null)) + .isInstanceOf(ConditionTimeoutException.class) + .hasMessageContaining("was not fulfilled within"); + + // Run the new fetch and wakeUp concurrently. Expect empty result from the blocked fetch. + ExecutorService executorService = + MoreExecutors.getExitingExecutorService( + (ThreadPoolExecutor) Executors.newFixedThreadPool(1)); + + AtomicBoolean gotEmptyResult = new AtomicBoolean(false); + executorService.execute( + () -> { + try { + gotEmptyResult.set(reader.fetch().nextSplit() == null); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + // Wakeup the reading of the split, and check the result size + reader.wakeUp(); + Awaitility.await().atMost(2, TimeUnit.SECONDS).until(() -> gotEmptyResult.get()); + + // Unpause and read the data in th next fetch + reader.pauseOrResumeSplits(ImmutableList.of(), ImmutableList.of(split)); Review Comment: after resuming the split from this line, shouldn't `wakeUp` be called? previous ` Awaitility.await()` was to make sure get return an empty result. -- 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