stevenzwu commented on code in PR #5967: URL: https://github.com/apache/iceberg/pull/5967#discussion_r1037741294
########## docs/flink-getting-started.md: ########## @@ -683,7 +683,47 @@ env.execute("Test Iceberg DataStream"); OVERWRITE and UPSERT can't be set together. In UPSERT mode, if the table is partitioned, the partition fields should be included in equality fields. {{< /hint >}} -## Write options +## Options +### Read options + +Flink read options are passed when configuring the FlinkSink, like this: + +``` +IcebergSource.forRowData() + .tableLoader(tableResource.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(scanContext.isStreaming()) + .streamingStartingStrategy(scanContext.streamingStartingStrategy()) + .startSnapshotTimestamp(scanContext.startSnapshotTimestamp()) + .startSnapshotId(scanContext.startSnapshotId()) + .set("monitor-interval", "10s") + .build() +``` +For Flink SQL, read options can be passed in via SQL hints like this: +``` +SELECT * FROM tableName /*+ OPTIONS('monitor-interval'='10s') */ +... +``` + +| Flink option | Default | Description | +| --------------------------- | ---------------------------------- | ------------------------------------------------------------ | +| snapshot-id | | For time travel in batch mode. Read data from the specified snapshot-id. | +| case-sensitive | false | Whether the sql is case sensitive | +| as-of-timestamp | | For time travel in batch mode. Read data from the most recent snapshot as of the given time in milliseconds. | +| starting-strategy | INCREMENTAL_FROM_LATEST_SNAPSHOT | Starting strategy for streaming execution. TABLE_SCAN_THEN_INCREMENTAL: Do a regular table scan then switch to the incremental mode. The incremental mode starts from the current snapshot exclusive. INCREMENTAL_FROM_LATEST_SNAPSHOT: Start incremental mode from the latest snapshot inclusive. If it is an empty map, all future append snapshots should be discovered. INCREMENTAL_FROM_EARLIEST_SNAPSHOT: Start incremental mode from the earliest snapshot inclusive. If it is an empty map, all future append snapshots should be discovered. INCREMENTAL_FROM_SNAPSHOT_ID: Start incremental mode from a snapshot with a specific id inclusive. INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: Start incremental mode from a snapshot with a specific timestamp inclusive. If the timestamp is between two snapshots, it should start from the snapshot after the timestamp. Just for FIP27 Source | +| start-snapshot-timestamp | | Start to read data from the most recent snapshot as of the given time in milliseconds. | +| start-snapshot-id | | Start to read data from the specified snapshot-id. | +| end-snapshot-id | The latest snapshot id | Specifies the end snapshot. | +| split-size | Table read.split.target-size | Overrides this table's read.split.target-size | +| split-lookback | Table read.split.planning-lookback | Overrides this table's read.split.planning-lookback | +| split-file-open-cost | Table read.split.open-file-cost | Overrides this table's read.split.open-file-cost | +| streaming | false | Sets whether the current task runs in streaming or batch mode. | +| monitor-interval | 10s | Interval for listening on the generation of new snapshots. | +| include-column-stats | false | Create a new scan from this that loads the column stats with each data file. Column stats include: value count, null value count, lower bounds, and upper bounds. | +| max-planning-snapshot-count | Integer.MAX_VALUE | If there are multiple new snapshots, configure the maximum number of snapshot forward at a time. | Review Comment: typically the split discovery interval should be <= upstream commit interval. when the job is provisioned properly for steady state, it shouldn't be able to process 10 snapshots/commits worth of data in one discovery cycle. that is like 10x of steady-state throughput, which seems like big over-provision. that is my intuition for default like 10. if the job is scaled up 10x for faster catch-up, that is where we can potentially have issue with default at 10. is it reasonable to ask users to change the `max-planning-snapshot-count` config in this case? -- 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