Content-Length: 409386 | pFad | http://github.com/apache/iceberg/pull/10207/commits/9b906aabc4b1f45cd8e445082197a411926c80f5

A4 Flink: Prevent setting endTag/endSnapshotId for streaming source by pvary · Pull Request #10207 · apache/iceberg · GitHub
Skip to content
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 5 commits into from
Apr 26, 2024
Merged
Show file tree
Hide file tree
Changes from 1 commit
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
Next Next commit
Flink: Prevent setting endTag/endSnapshotId for streaming source
  • Loading branch information
Peter Vary committed Apr 23, 2024
commit 9b906aabc4b1f45cd8e445082197a411926c80f5
Original file line number Diff line number Diff line change
Expand Up @@ -155,6 +155,13 @@ private void validate() {
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(
Copy link
Contributor

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.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Here is the code:

      Snapshot toSnapshotInclusive =
          toSnapshotInclusive(
              lastConsumedSnapshotId, currentSnapshot, scanContext.maxPlanningSnapshotCount());
      IcebergEnumeratorPosition newPosition =
          IcebergEnumeratorPosition.of(
              toSnapshotInclusive.snapshotId(), toSnapshotInclusive.timestampMillis());
      ScanContext incrementalScan =
          scanContext.copyWithAppendsBetween(
              lastPosition.snapshotId(), toSnapshotInclusive.snapshotId());

The toSnapshotInclusive reads until the current snapshot. Its only role is to prevent reading more snapshot than maxPlanningSnapshotCount. And we set this as a last snapshot for the scan.

Copy link
Contributor

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).

Copy link
Contributor Author

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.

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(
Expand Down
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);
}
}
Loading








ApplySandwichStrip

pFad - (p)hone/(F)rame/(a)nonymizer/(d)eclutterfier!      Saves Data!


--- a PPN by Garber Painting Akron. With Image Size Reduction included!

Fetched URL: http://github.com/apache/iceberg/pull/10207/commits/9b906aabc4b1f45cd8e445082197a411926c80f5

Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy