Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@
import java.util.UUID;
import java.util.function.Function;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.SupportsConcurrentExecutionAttempts;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.api.common.functions.MapFunction;
import org.apache.flink.api.common.typeinfo.TypeInformation;
Expand Down Expand Up @@ -139,7 +140,8 @@ public class IcebergSink
SupportsPreWriteTopology<RowData>,
SupportsCommitter<IcebergCommittable>,
SupportsPreCommitTopology<WriteResult, IcebergCommittable>,
SupportsPostCommitTopology<IcebergCommittable> {
SupportsPostCommitTopology<IcebergCommittable>,
SupportsConcurrentExecutionAttempts {
private static final Logger LOG = LoggerFactory.getLogger(IcebergSink.class);
private final TableLoader tableLoader;
private final Map<String, String> snapshotProperties;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,19 +44,24 @@
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.Parameter;
import org.apache.iceberg.ParameterizedTestExtension;
import org.apache.iceberg.Parameters;
import org.apache.iceberg.flink.FlinkConfigOptions;
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.TestTemplate;
import org.junit.jupiter.api.Timeout;
import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.api.extension.RegisterExtension;

/**
* There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case
* anything goes wrong unexpectedly.
*/
@Timeout(value = 60)
@ExtendWith(ParameterizedTestExtension.class)
public class TestIcebergSpeculativeExecutionSupport extends TestBase {
private static final int NUM_TASK_MANAGERS = 1;
private static final int NUM_TASK_SLOTS = 3;
Expand Down Expand Up @@ -90,6 +95,14 @@ protected TableEnvironment getTableEnv() {
return tEnv;
}

@Parameter(index = 0)
private boolean useV2Sink;

@Parameters(name = "useV2Sink = {0}")
public static Object[][] parameters() {
return new Object[][] {{true}, {false}};
}

@BeforeEach
public void before() throws IOException {
String warehouse =
Expand All @@ -114,8 +127,9 @@ public void after() {
dropCatalog(CATALOG_NAME, true);
}

@Test
@TestTemplate
public void testSpeculativeExecution() throws Exception {
tEnv.getConfig().set("table.exec.iceberg.use-v2-sink", String.valueOf(useV2Sink));
Table table =
tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME));
DataStream<Row> slowStream =
Expand Down