-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Flink: Prevent setting endTag/endSnapshotId for streaming source #10207
Merged
Merged
Changes from 1 commit
Commits
Show all changes
5 commits
Select commit
Hold shift + click to select a range
9b906aa
Flink: Prevent setting endTag/endSnapshotId for streaming source
c2815f6
More concrete error message test
6d03fa6
Skip validation for ScanContexts created by the monitor functions, an…
e247e82
Validate only at source creation
789191a
Fixed validation for FlinkSource too, and added tests for bounded sou…
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next
Next commit
Flink: Prevent setting endTag/endSnapshotId for streaming source
- Loading branch information
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
121 changes: 121 additions & 0 deletions
121
flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,121 @@ | ||
/* | ||
* 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() { | ||
Assertions.assertThatThrownBy( | ||
() -> | ||
ScanContext.builder() | ||
.streaming(true) | ||
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) | ||
.build()) | ||
.hasMessageContaining("Invalid starting snapshot id for") | ||
pvary marked this conversation as resolved.
Show resolved
Hide resolved
|
||
.isInstanceOf(IllegalArgumentException.class); | ||
|
||
Assertions.assertThatThrownBy( | ||
() -> | ||
ScanContext.builder() | ||
.streaming(true) | ||
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) | ||
.startSnapshotId(1L) | ||
.startSnapshotTimestamp(1L) | ||
.build()) | ||
.hasMessageContaining("Invalid starting snapshot timestamp for") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
} | ||
|
||
@Test | ||
void testIncrementalFromSnapshotTimestamp() { | ||
Assertions.assertThatThrownBy( | ||
() -> | ||
ScanContext.builder() | ||
.streaming(true) | ||
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) | ||
.build()) | ||
.hasMessageContaining("Invalid starting snapshot timestamp for") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
|
||
Assertions.assertThatThrownBy( | ||
() -> | ||
ScanContext.builder() | ||
.streaming(true) | ||
.startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) | ||
.startSnapshotId(1L) | ||
.startSnapshotTimestamp(1L) | ||
.build()) | ||
.hasMessageContaining("Invalid starting snapshot id for") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
} | ||
|
||
@Test | ||
void testStreaming() { | ||
Assertions.assertThatThrownBy(() -> ScanContext.builder().streaming(true).useTag("tag").build()) | ||
.hasMessageContaining("Cannot scan table using ref") | ||
.hasMessageContaining("for streaming reader") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
|
||
Assertions.assertThatThrownBy( | ||
() -> ScanContext.builder().streaming(true).useSnapshotId(1L).build()) | ||
.hasMessageContaining("Cannot set snapshot-id option for streaming reader") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
|
||
Assertions.assertThatThrownBy( | ||
() -> ScanContext.builder().streaming(true).asOfTimestamp(1L).build()) | ||
.hasMessageContaining("Cannot set as-of-timestamp option for streaming reader") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
|
||
Assertions.assertThatThrownBy( | ||
() -> ScanContext.builder().streaming(true).endSnapshotId(1L).build()) | ||
.hasMessageContaining("Cannot set end-snapshot-id option for streaming reader") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
|
||
Assertions.assertThatThrownBy(() -> ScanContext.builder().streaming(true).endTag("tag").build()) | ||
.hasMessageContaining("Cannot set end-tag option for streaming reader") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
} | ||
|
||
@Test | ||
void testStartConflict() { | ||
Assertions.assertThatThrownBy( | ||
() -> ScanContext.builder().startTag("tag").startSnapshotId(1L).build()) | ||
.hasMessageContaining("START_SNAPSHOT_ID and START_TAG cannot both be set.") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
} | ||
|
||
@Test | ||
void testEndConflict() { | ||
Assertions.assertThatThrownBy( | ||
() -> ScanContext.builder().endTag("tag").endSnapshotId(1L).build()) | ||
.hasMessageContaining("END_SNAPSHOT_ID and END_TAG cannot both be set.") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
} | ||
|
||
@Test | ||
void testMaxAllowedPlanningFailures() { | ||
Assertions.assertThatThrownBy( | ||
() -> ScanContext.builder().maxAllowedPlanningFailures(-2).build()) | ||
.hasMessageContaining("annot set maxAllowedPlanningFailures to a negative number") | ||
.isInstanceOf(IllegalArgumentException.class); | ||
} | ||
} |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
where were end snapshot (id or tag) silently overridden? I thought it might be a valid scenario. E.g., during backfill, maybe it can be streaming mode and discover splits snapshot by snapshot with an end snapshot.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
iceberg/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java
Line 131 in fbcd142
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Here is the code:
The
toSnapshotInclusive
reads until the current snapshot. Its only role is to prevent reading more snapshot thanmaxPlanningSnapshotCount
. And we set this as a last snapshot for the scan.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
this is to set the end snapshot per incremental scan/discovery. The source doesn't check/support end snapshot (like Kafka source's bounded end position).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, I got it. This is why we need the new validations to prevent the wrong parametrization of the source.