stevenzwu commented on code in PR #10832: URL: https://github.com/apache/iceberg/pull/10832#discussion_r1716294343
########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java: ########## @@ -205,12 +225,35 @@ private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumer // Only do scan planning if nothing is restored from checkpoint state List<IcebergSourceSplit> splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + boolean shouldInferParallelism() { Review Comment: good catch. it was package visible as earlier implementation does the inferring in the `IcebergTableSource`. with the refactoring and `buildStream` introduced. this can be private. ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java: ########## @@ -79,6 +80,12 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { @TempDir protected Path temporaryFolder; private static final InMemoryReporter REPORTER = InMemoryReporter.createWithRetainedMetrics(); + public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false) + // disable inferring source parallelism Review Comment: actually, we don't need the disabling for this particular test as it doesn't go through the `buildStream(env)` path where infer parallelism happens. will revert the change ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java: ########## @@ -545,5 +579,66 @@ public IcebergSource<T> build() { table, emitter); } + + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. + * + * @return data stream from the Iceberg source + */ + public DataStream<T> buildStream(StreamExecutionEnvironment env) { Review Comment: currently, `outputTypeInfo` can be inferred from the `ReaderFunction` if using provided RowData or Avro reader. ``` if (outputTypeInfo == null) { this.outputTypeInfo = inferOutputTypeInfo(table, context, readerFunction); } ``` `watermarkStrategy` is defaulted `WatermarkStrategy.noWatermarks`. so it is not mandatory either. ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java: ########## @@ -503,28 +569,10 @@ public IcebergSource<T> build() { new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); } - ScanContext context = contextBuilder.build(); + this.context = contextBuilder.build(); Review Comment: agree side-effect is undesirable. let me think of a way to refactor the code. maybe extract the `ScanContext` building into a separate method from `build()` ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java: ########## @@ -95,6 +103,8 @@ public class IcebergSource<T> implements Source<T, IcebergSourceSplit, IcebergEn private final SerializableRecordEmitter<T> emitter; private final String tableName; + private volatile List<IcebergSourceSplit> batchSplits; Review Comment: `planSplitsForBatch` can be called twice: (1) source/stream construction by main thread (2) enumerator creation. confirmed via debugger they are done by different threads. (1) from main thread <img width="575" alt="image" src="https://github.com/user-attachments/assets/eac07c33-8ee3-4648-ac2d-2812ac0dcb44"> (2) from actor thread <img width="633" alt="image" src="https://github.com/user-attachments/assets/1e296e28-1f5f-4465-8685-4f128cca2f2d"> ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java: ########## @@ -464,6 +519,15 @@ public Builder<T> watermarkColumnTimeUnit(TimeUnit timeUnit) { return this; } + /** + * Optional. Default is no watermark strategy. Only relevant if using the {@link + * Builder#buildStream(StreamExecutionEnvironment)}. + */ + public Builder<T> watermarkStrategy(WatermarkStrategy<T> newStrategy) { Review Comment: this is only for the `buildStream` method. this just moved the watermark strategy from `env.fromSource` to the builder. for regular `build` method, users would also need to set the watermark strategy, which most likely would be none. ``` DataStream<RowData> stream = env.fromSource( sourceBuilder().build(), WatermarkStrategy.noWatermarks(), "IcebergSource", TypeInformation.of(RowData.class)); ``` ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java: ########## @@ -0,0 +1,171 @@ +/* + * 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.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +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.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List<Record> batch1 = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch1); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List<Record> batch1 = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch1); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 3); + + DataStream<Row> dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector<Row> collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator<Row> iterator = collector.getOutput()) { + List<Row> result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = Review Comment: I just tried it with debugger. the value is the default parallelism of 4 while the expected inferred source parallelism is 1 after the `executeAsync()` ``` DataStream<Row> dataStream = IcebergSource.forRowData() .tableLoader(CATALOG_EXTENSION.tableLoader()) .table(table) .flinkConfig(config) // force one file per split .splitSize(1L) .buildStream(env) .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); int sourceParallelism = env.getTransformations().get(0).getParallelism(); ``` ########## flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java: ########## @@ -205,12 +225,35 @@ private SplitEnumerator<IcebergSourceSplit, IcebergEnumeratorState> createEnumer // Only do scan planning if nothing is restored from checkpoint state List<IcebergSourceSplit> splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { Review Comment: good catch. it was package visible as earlier implementation does the inferring in the `IcebergTableSource`. with the refactoring and `buildStream` introduced. this can be private. ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java: ########## @@ -0,0 +1,181 @@ +/* + * 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.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +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.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List<Record> batch1 = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch1); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List<Record> batch1 = RandomGenericData.generate(table.schema(), 1, 0); Review Comment: fixed ########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java: ########## @@ -0,0 +1,181 @@ +/* + * 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.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +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.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List<Record> batch1 = RandomGenericData.generate(table.schema(), 1, 0); Review Comment: fixed -- 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