This is an automated email from the ASF dual-hosted git repository.
stevenwu pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/iceberg.git
The following commit(s) were added to refs/heads/main by this push:
new 1e35bf96ec Flink: Backport #10207 to v1.18 and v1.17 (#10235)
1e35bf96ec is described below
commit 1e35bf96ecacd5c5175116f40fa3e097991d04d2
Author: pvary <[email protected]>
AuthorDate: Sat Apr 27 15:44:39 2024 +0200
Flink: Backport #10207 to v1.18 and v1.17 (#10235)
Co-authored-by: Peter Vary <[email protected]>
---
.../apache/iceberg/flink/source/FlinkSource.java | 9 +-
.../apache/iceberg/flink/source/IcebergSource.java | 1 +
.../apache/iceberg/flink/source/ScanContext.java | 11 +-
.../iceberg/flink/source/TestFlinkInputFormat.java | 20 ++++
.../flink/source/TestIcebergSourceBounded.java | 20 ++++
.../flink/source/TestIcebergSourceContinuous.java | 15 +++
.../iceberg/flink/source/TestScanContext.java | 111 +++++++++++++++++++++
.../iceberg/flink/source/TestStreamScanSql.java | 2 +
.../apache/iceberg/flink/source/FlinkSource.java | 9 +-
.../apache/iceberg/flink/source/IcebergSource.java | 1 +
.../apache/iceberg/flink/source/ScanContext.java | 11 +-
.../iceberg/flink/source/TestFlinkInputFormat.java | 20 ++++
.../flink/source/TestIcebergSourceBounded.java | 20 ++++
.../flink/source/TestIcebergSourceContinuous.java | 15 +++
.../iceberg/flink/source/TestScanContext.java | 111 +++++++++++++++++++++
.../iceberg/flink/source/TestStreamScanSql.java | 2 +
16 files changed, 360 insertions(+), 18 deletions(-)
diff --git
a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
index fa1656c552..b1431a32dd 100644
---
a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
+++
b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
@@ -45,12 +45,8 @@ import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.PropertyUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
public class FlinkSource {
- private static final Logger LOG = LoggerFactory.getLogger(FlinkSource.class);
-
private FlinkSource() {}
/**
@@ -263,8 +259,9 @@ public class FlinkSource {
contextBuilder.resolveConfig(table, readOptions, readableConfig);
- return new FlinkInputFormat(
- tableLoader, icebergSchema, io, encryption, contextBuilder.build());
+ ScanContext context = contextBuilder.build();
+ context.validate();
+ return new FlinkInputFormat(tableLoader, icebergSchema, io, encryption,
context);
}
public DataStream<RowData> build() {
diff --git
a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
index 8ec92c8c2c..48201ea093 100644
---
a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
+++
b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
@@ -504,6 +504,7 @@ public class IcebergSource<T> implements Source<T,
IcebergSourceSplit, IcebergEn
}
ScanContext context = contextBuilder.build();
+ context.validate();
if (readerFunction == null) {
if (table instanceof BaseMetadataTable) {
MetaDataReaderFunction rowDataReaderFunction =
diff --git
a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
index cf57a126ae..ab79a31739 100644
---
a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
+++
b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
@@ -129,11 +129,9 @@ public class ScanContext implements Serializable {
this.maxAllowedPlanningFailures = maxAllowedPlanningFailures;
this.watermarkColumn = watermarkColumn;
this.watermarkColumnTimeUnit = watermarkColumnTimeUnit;
-
- validate();
}
- private void validate() {
+ void validate() {
if (isStreaming) {
if (startingStrategy ==
StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) {
Preconditions.checkArgument(
@@ -155,6 +153,13 @@ public class ScanContext implements Serializable {
Preconditions.checkArgument(
tag == null,
String.format("Cannot scan table using ref %s configured for
streaming reader", tag));
+ Preconditions.checkArgument(
+ snapshotId == null, "Cannot set snapshot-id option for streaming
reader");
+ Preconditions.checkArgument(
+ asOfTimestamp == null, "Cannot set as-of-timestamp option for
streaming reader");
+ Preconditions.checkArgument(
+ endSnapshotId == null, "Cannot set end-snapshot-id option for
streaming reader");
+ Preconditions.checkArgument(endTag == null, "Cannot set end-tag option
for streaming reader");
}
Preconditions.checkArgument(
diff --git
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
index ed3f54bec6..366a3c2b50 100644
---
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
+++
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
@@ -18,13 +18,16 @@
*/
package org.apache.iceberg.flink.source;
+import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA;
import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.assertj.core.api.Assumptions.assumeThat;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.types.logical.RowType;
@@ -184,6 +187,23 @@ public class TestFlinkInputFormat extends TestFlinkSource {
TestHelpers.assertRows(result, expected);
}
+ @TestTemplate
+ public void testValidation() {
+ catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER,
SCHEMA);
+
+ assertThatThrownBy(
+ () ->
+ FlinkSource.forRowData()
+ .env(StreamExecutionEnvironment.getExecutionEnvironment())
+ .tableLoader(tableLoader())
+ .streaming(false)
+ .endTag("tag")
+ .endSnapshotId(1L)
+ .build())
+ .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.")
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+
private List<Row> runFormat(FlinkInputFormat inputFormat) throws IOException
{
RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema());
return TestHelpers.readRows(inputFormat, rowType);
diff --git
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java
index 3c0c38e111..0bf09b1643 100644
---
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java
+++
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java
@@ -18,6 +18,9 @@
*/
package org.apache.iceberg.flink.source;
+import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -42,8 +45,25 @@ import org.apache.iceberg.flink.data.RowDataToRowMapper;
import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.junit.jupiter.api.TestTemplate;
public class TestIcebergSourceBounded extends TestFlinkScan {
+ @TestTemplate
+ public void testValidation() {
+ catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER,
SCHEMA);
+
+ assertThatThrownBy(
+ () ->
+ IcebergSource.forRowData()
+ .tableLoader(tableLoader())
+ .assignerFactory(new SimpleSplitAssignerFactory())
+ .streaming(false)
+ .endTag("tag")
+ .endSnapshotId(1L)
+ .build())
+ .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.")
+ .isInstanceOf(IllegalArgumentException.class);
+ }
@Override
protected List<Row> runWithProjection(String... projected) throws Exception {
diff --git
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java
index bfd7fa5758..3f5af78704 100644
---
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java
+++
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java
@@ -19,6 +19,7 @@
package org.apache.iceberg.flink.source;
import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
import java.time.Duration;
import java.util.Collection;
@@ -456,6 +457,20 @@ public class TestIcebergSourceContinuous {
}
}
+ @Test
+ public void testValidation() {
+ assertThatThrownBy(
+ () ->
+ IcebergSource.forRowData()
+ .tableLoader(tableResource.tableLoader())
+ .assignerFactory(new SimpleSplitAssignerFactory())
+ .streaming(true)
+ .endTag("tag")
+ .build())
+ .hasMessage("Cannot set end-tag option for streaming reader")
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+
private DataStream<Row> createStream(ScanContext scanContext) throws
Exception {
// start the source and collect output
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
diff --git
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java
new file mode 100644
index 0000000000..146f192625
--- /dev/null
+++
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java
@@ -0,0 +1,111 @@
+/*
+ * 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 org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+class TestScanContext {
+ @Test
+ void testIncrementalFromSnapshotId() {
+ ScanContext context =
+ ScanContext.builder()
+ .streaming(true)
+
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID)
+ .build();
+ assertException(
+ context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID
strategy: null");
+
+ context =
+ ScanContext.builder()
+ .streaming(true)
+
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID)
+ .startSnapshotId(1L)
+ .startSnapshotTimestamp(1L)
+ .build();
+ assertException(
+ context,
+ "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID
strategy: not null");
+ }
+
+ @Test
+ void testIncrementalFromSnapshotTimestamp() {
+ ScanContext context =
+ ScanContext.builder()
+ .streaming(true)
+
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP)
+ .build();
+ assertException(
+ context,
+ "Invalid starting snapshot timestamp for
SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null");
+
+ context =
+ ScanContext.builder()
+ .streaming(true)
+
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP)
+ .startSnapshotId(1L)
+ .startSnapshotTimestamp(1L)
+ .build();
+ assertException(
+ context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID
strategy: not null");
+ }
+
+ @Test
+ void testStreaming() {
+ ScanContext context =
ScanContext.builder().streaming(true).useTag("tag").build();
+ assertException(context, "Cannot scan table using ref tag configured for
streaming reader");
+
+ context = ScanContext.builder().streaming(true).useSnapshotId(1L).build();
+ assertException(context, "Cannot set snapshot-id option for streaming
reader");
+
+ context = ScanContext.builder().streaming(true).asOfTimestamp(1L).build();
+ assertException(context, "Cannot set as-of-timestamp option for streaming
reader");
+
+ context = ScanContext.builder().streaming(true).endSnapshotId(1L).build();
+ assertException(context, "Cannot set end-snapshot-id option for streaming
reader");
+
+ context = ScanContext.builder().streaming(true).endTag("tag").build();
+ assertException(context, "Cannot set end-tag option for streaming reader");
+ }
+
+ @Test
+ void testStartConflict() {
+ ScanContext context =
ScanContext.builder().startTag("tag").startSnapshotId(1L).build();
+ assertException(context, "START_SNAPSHOT_ID and START_TAG cannot both be
set.");
+ }
+
+ @Test
+ void testEndConflict() {
+ ScanContext context =
ScanContext.builder().endTag("tag").endSnapshotId(1L).build();
+ assertException(context, "END_SNAPSHOT_ID and END_TAG cannot both be
set.");
+ }
+
+ @Test
+ void testMaxAllowedPlanningFailures() {
+ ScanContext context =
ScanContext.builder().maxAllowedPlanningFailures(-2).build();
+ assertException(
+ context, "Cannot set maxAllowedPlanningFailures to a negative number
other than -1.");
+ }
+
+ private void assertException(ScanContext context, String message) {
+ Assertions.assertThatThrownBy(() -> context.validate())
+ .hasMessage(message)
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+}
diff --git
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java
index 9e043bbbbb..4214984646 100644
---
a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java
+++
b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java
@@ -48,7 +48,9 @@ import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.Timeout;
+@Timeout(60)
public class TestStreamScanSql extends CatalogTestBase {
private static final String TABLE = "test_table";
private static final FileFormat FORMAT = FileFormat.PARQUET;
diff --git
a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
index fa1656c552..b1431a32dd 100644
---
a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
+++
b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
@@ -45,12 +45,8 @@ import org.apache.iceberg.io.FileIO;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.PropertyUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
public class FlinkSource {
- private static final Logger LOG = LoggerFactory.getLogger(FlinkSource.class);
-
private FlinkSource() {}
/**
@@ -263,8 +259,9 @@ public class FlinkSource {
contextBuilder.resolveConfig(table, readOptions, readableConfig);
- return new FlinkInputFormat(
- tableLoader, icebergSchema, io, encryption, contextBuilder.build());
+ ScanContext context = contextBuilder.build();
+ context.validate();
+ return new FlinkInputFormat(tableLoader, icebergSchema, io, encryption,
context);
}
public DataStream<RowData> build() {
diff --git
a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
index 8ec92c8c2c..48201ea093 100644
---
a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
+++
b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java
@@ -504,6 +504,7 @@ public class IcebergSource<T> implements Source<T,
IcebergSourceSplit, IcebergEn
}
ScanContext context = contextBuilder.build();
+ context.validate();
if (readerFunction == null) {
if (table instanceof BaseMetadataTable) {
MetaDataReaderFunction rowDataReaderFunction =
diff --git
a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
index cf57a126ae..ab79a31739 100644
---
a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
+++
b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
@@ -129,11 +129,9 @@ public class ScanContext implements Serializable {
this.maxAllowedPlanningFailures = maxAllowedPlanningFailures;
this.watermarkColumn = watermarkColumn;
this.watermarkColumnTimeUnit = watermarkColumnTimeUnit;
-
- validate();
}
- private void validate() {
+ void validate() {
if (isStreaming) {
if (startingStrategy ==
StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) {
Preconditions.checkArgument(
@@ -155,6 +153,13 @@ public class ScanContext implements Serializable {
Preconditions.checkArgument(
tag == null,
String.format("Cannot scan table using ref %s configured for
streaming reader", tag));
+ Preconditions.checkArgument(
+ snapshotId == null, "Cannot set snapshot-id option for streaming
reader");
+ Preconditions.checkArgument(
+ asOfTimestamp == null, "Cannot set as-of-timestamp option for
streaming reader");
+ Preconditions.checkArgument(
+ endSnapshotId == null, "Cannot set end-snapshot-id option for
streaming reader");
+ Preconditions.checkArgument(endTag == null, "Cannot set end-tag option
for streaming reader");
}
Preconditions.checkArgument(
diff --git
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
index ed3f54bec6..366a3c2b50 100644
---
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
+++
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
@@ -18,13 +18,16 @@
*/
package org.apache.iceberg.flink.source;
+import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA;
import static org.apache.iceberg.types.Types.NestedField.required;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
import static org.assertj.core.api.Assumptions.assumeThat;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableSchema;
import org.apache.flink.table.types.logical.RowType;
@@ -184,6 +187,23 @@ public class TestFlinkInputFormat extends TestFlinkSource {
TestHelpers.assertRows(result, expected);
}
+ @TestTemplate
+ public void testValidation() {
+ catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER,
SCHEMA);
+
+ assertThatThrownBy(
+ () ->
+ FlinkSource.forRowData()
+ .env(StreamExecutionEnvironment.getExecutionEnvironment())
+ .tableLoader(tableLoader())
+ .streaming(false)
+ .endTag("tag")
+ .endSnapshotId(1L)
+ .build())
+ .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.")
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+
private List<Row> runFormat(FlinkInputFormat inputFormat) throws IOException
{
RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema());
return TestHelpers.readRows(inputFormat, rowType);
diff --git
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java
index 3c0c38e111..0bf09b1643 100644
---
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java
+++
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java
@@ -18,6 +18,9 @@
*/
package org.apache.iceberg.flink.source;
+import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
import java.util.Collections;
import java.util.List;
import java.util.Map;
@@ -42,8 +45,25 @@ import org.apache.iceberg.flink.data.RowDataToRowMapper;
import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.junit.jupiter.api.TestTemplate;
public class TestIcebergSourceBounded extends TestFlinkScan {
+ @TestTemplate
+ public void testValidation() {
+ catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER,
SCHEMA);
+
+ assertThatThrownBy(
+ () ->
+ IcebergSource.forRowData()
+ .tableLoader(tableLoader())
+ .assignerFactory(new SimpleSplitAssignerFactory())
+ .streaming(false)
+ .endTag("tag")
+ .endSnapshotId(1L)
+ .build())
+ .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.")
+ .isInstanceOf(IllegalArgumentException.class);
+ }
@Override
protected List<Row> runWithProjection(String... projected) throws Exception {
diff --git
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java
index 61e05e99e1..5765b73a1f 100644
---
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java
+++
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java
@@ -19,6 +19,7 @@
package org.apache.iceberg.flink.source;
import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
import java.time.Duration;
import java.util.Collection;
@@ -473,6 +474,20 @@ public class TestIcebergSourceContinuous {
}
}
+ @Test
+ public void testValidation() {
+ assertThatThrownBy(
+ () ->
+ IcebergSource.forRowData()
+ .tableLoader(tableResource.tableLoader())
+ .assignerFactory(new SimpleSplitAssignerFactory())
+ .streaming(true)
+ .endTag("tag")
+ .build())
+ .hasMessage("Cannot set end-tag option for streaming reader")
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+
private DataStream<Row> createStream(ScanContext scanContext) throws
Exception {
// start the source and collect output
StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
diff --git
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java
new file mode 100644
index 0000000000..146f192625
--- /dev/null
+++
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java
@@ -0,0 +1,111 @@
+/*
+ * 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 org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+class TestScanContext {
+ @Test
+ void testIncrementalFromSnapshotId() {
+ ScanContext context =
+ ScanContext.builder()
+ .streaming(true)
+
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID)
+ .build();
+ assertException(
+ context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID
strategy: null");
+
+ context =
+ ScanContext.builder()
+ .streaming(true)
+
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID)
+ .startSnapshotId(1L)
+ .startSnapshotTimestamp(1L)
+ .build();
+ assertException(
+ context,
+ "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID
strategy: not null");
+ }
+
+ @Test
+ void testIncrementalFromSnapshotTimestamp() {
+ ScanContext context =
+ ScanContext.builder()
+ .streaming(true)
+
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP)
+ .build();
+ assertException(
+ context,
+ "Invalid starting snapshot timestamp for
SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null");
+
+ context =
+ ScanContext.builder()
+ .streaming(true)
+
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP)
+ .startSnapshotId(1L)
+ .startSnapshotTimestamp(1L)
+ .build();
+ assertException(
+ context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID
strategy: not null");
+ }
+
+ @Test
+ void testStreaming() {
+ ScanContext context =
ScanContext.builder().streaming(true).useTag("tag").build();
+ assertException(context, "Cannot scan table using ref tag configured for
streaming reader");
+
+ context = ScanContext.builder().streaming(true).useSnapshotId(1L).build();
+ assertException(context, "Cannot set snapshot-id option for streaming
reader");
+
+ context = ScanContext.builder().streaming(true).asOfTimestamp(1L).build();
+ assertException(context, "Cannot set as-of-timestamp option for streaming
reader");
+
+ context = ScanContext.builder().streaming(true).endSnapshotId(1L).build();
+ assertException(context, "Cannot set end-snapshot-id option for streaming
reader");
+
+ context = ScanContext.builder().streaming(true).endTag("tag").build();
+ assertException(context, "Cannot set end-tag option for streaming reader");
+ }
+
+ @Test
+ void testStartConflict() {
+ ScanContext context =
ScanContext.builder().startTag("tag").startSnapshotId(1L).build();
+ assertException(context, "START_SNAPSHOT_ID and START_TAG cannot both be
set.");
+ }
+
+ @Test
+ void testEndConflict() {
+ ScanContext context =
ScanContext.builder().endTag("tag").endSnapshotId(1L).build();
+ assertException(context, "END_SNAPSHOT_ID and END_TAG cannot both be
set.");
+ }
+
+ @Test
+ void testMaxAllowedPlanningFailures() {
+ ScanContext context =
ScanContext.builder().maxAllowedPlanningFailures(-2).build();
+ assertException(
+ context, "Cannot set maxAllowedPlanningFailures to a negative number
other than -1.");
+ }
+
+ private void assertException(ScanContext context, String message) {
+ Assertions.assertThatThrownBy(() -> context.validate())
+ .hasMessage(message)
+ .isInstanceOf(IllegalArgumentException.class);
+ }
+}
diff --git
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java
index 9e043bbbbb..4214984646 100644
---
a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java
+++
b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java
@@ -48,7 +48,9 @@ import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.TestTemplate;
+import org.junit.jupiter.api.Timeout;
+@Timeout(60)
public class TestStreamScanSql extends CatalogTestBase {
private static final String TABLE = "test_table";
private static final FileFormat FORMAT = FileFormat.PARQUET;