venkata91 commented on code in PR #10548: URL: https://github.com/apache/iceberg/pull/10548#discussion_r1686993453
########## flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpecExecSupport.java: ########## @@ -0,0 +1,179 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SlowTaskDetectorOptions; +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.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.TestBase; +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; + +public class TestIcebergSpecExecSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .setConfiguration(configure()) + .build()); + + private StreamTableEnvironment tEnv; + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String INPUT_TABLE_NAME = "test_table"; + private static final String OUTPUT_TABLE_NAME = "sink_table"; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + } + } + return tEnv; + } + + @BeforeEach + public void before() throws IOException { + String warehouse = + String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); + sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); + sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); + dropDatabase(DATABASE_NAME, true); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testSpeculativeExecution() throws Exception { Review Comment: Nice catch! Looks like I didn't set `table.exec.iceberg.use-flip27-source` to `true`. With this, without the changes in `AbstractIcebergEnumerator`, it will fail with an exception saying: ``` The split enumerator StaticIcebergEnumerator must implement SupportsHandleExecutionAttemptSourceEvent to be used in concurrent execution attempts scenario (e.g. if speculative execution is enabled" ``` -- 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