Guosmilesmile commented on code in PR #13853: URL: https://github.com/apache/iceberg/pull/13853#discussion_r2285564190
########## docs/docs/flink-table-maintenance.md: ########## @@ -0,0 +1,361 @@ +--- +title: "Flink Table Maintenance " +--- +<!-- + - 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. + --> + +# Flink Table Maintenance + +## Overview + +In **Apache Iceberg** deployments within **Flink streaming environments**, implementing automated table maintenance operations—including `snapshot expiration`, `small file compaction`, and `orphan file cleanup`—is critical for optimal query performance and storage efficiency. + +Traditionally, these maintenance operations were exclusively accessible through **Iceberg Spark Actions**, necessitating the deployment and management of dedicated Spark clusters. This dependency on **Spark infrastructure** solely for table optimization introduces significant **architectural complexity** and **operational overhead**. + +The `TableMaintenance` API in **Apache Iceberg** empowers **Flink jobs** to execute maintenance tasks **natively**, either embedded within existing streaming pipelines or deployed as standalone Flink jobs. This eliminates dependencies on external systems, thereby **streamlining architecture**, **reducing operational costs**, and **enhancing automation capabilities**. + +## Supported Features (Flink) + +### ExpireSnapshots +Removes old snapshots and their files. Internally uses `cleanExpiredFiles(true)` when committing, so expired metadata/files are cleaned up automatically. + +```java +.add(ExpireSnapshots.builder() + .maxSnapshotAge(Duration.ofDays(7)) + .retainLast(10) + .deleteBatchSize(1000)) +``` + +### RewriteDataFiles +Compacts small files to optimize file sizes. Supports partial progress commits and limiting maximum rewritten bytes per run. + +```java +.add(RewriteDataFiles.builder() + .targetFileSizeBytes(256 * 1024 * 1024) + .minFileSizeBytes(32 * 1024 * 1024) + .partialProgressEnabled(true) + .partialProgressMaxCommits(5)) +``` + +## Lock Management + +The `TriggerLockFactory` is essential for coordinating maintenance tasks. It prevents concurrent maintenance operations on the same table, which could lead to conflicts or data corruption. This locking mechanism is necessary even for a single job, as multiple instances of the same task could otherwise conflict. + +### Why Locks Are Needed +- **Concurrent Access**: Multiple Flink jobs may attempt maintenance simultaneously +- **Data Consistency**: Ensures only one maintenance operation runs per table at a time +- **Resource Management**: Prevents resource conflicts and scheduling issues +- **Avoid Duplicate Work**: Even when only a single compaction job is scheduled, multiple instances could attempt the same operation, leading to redundant work and wasted resources. + +### Supported Lock Types + +#### JDBC Lock Factory +Uses a database table to manage distributed locks: + +```java +Map<String, String> jdbcProps = new HashMap<>(); +jdbcProps.put("jdbc.user", "flink"); +jdbcProps.put("jdbc.password", "flinkpw"); + +TriggerLockFactory lockFactory = new JdbcLockFactory( + "jdbc:postgresql://localhost:5432/iceberg", // JDBC URL + "catalog.db.table", // Lock ID (unique identifier) + jdbcProps // JDBC connection properties +); +``` + +#### ZooKeeper Lock Factory +Uses Apache ZooKeeper for distributed coordination: + +```java +TriggerLockFactory lockFactory = new ZkLockFactory( + "localhost:2181", // ZooKeeper connection string + "catalog.db.table", // Lock ID (unique identifier) + 60000, // sessionTimeoutMs + 15000, // connectionTimeoutMs + 3000, // baseSleepTimeMs + 3 // maxRetries +); +``` + +## Quick Start + +The following example demonstrates the implementation of automated maintenance for an Iceberg table within a Flink environment. + +```java +StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); +TableLoader tableLoader = TableLoader.fromHadoopTable("path/to/table"); + +Map<String, String> jdbcProps = new HashMap<>(); +jdbcProps.put("jdbc.user", "flink"); +jdbcProps.put("jdbc.password", "flinkpw"); + +// JdbcLockFactory Example +TriggerLockFactory lockFactory = new JdbcLockFactory( + "jdbc:postgresql://localhost:5432/iceberg", // JDBC URL + "catalog.db.table", // Lock ID (unique identifier) + jdbcProps // JDBC connection properties +); + +TableMaintenance.forTable(env, tableLoader, lockFactory) + .uidSuffix("my-maintenance-job") + .rateLimit(Duration.ofMinutes(10)) + .lockCheckDelay(Duration.ofSeconds(10)) + .add(ExpireSnapshots.builder() + .scheduleOnCommitCount(10) + .maxSnapshotAge(Duration.ofMinutes(10)) + .retainLast(5) + .deleteBatchSize(5) + .parallelism(8)) + .add(RewriteDataFiles.builder() + .scheduleOnDataFileCount(10) + .targetFileSizeBytes(128 * 1024 * 1024) + .partialProgressEnabled(true) + .partialProgressMaxCommits(10)) + .append(); + +env.execute("Table Maintenance Job"); +``` + +## Configuration Options + +### TableMaintenance Builder + +| Method | Description | Default | +|--------|-------------|---------| +| `uidSuffix(String)` | Unique identifier suffix for the job | Random UUID | +| `rateLimit(Duration)` | Minimum interval between task executions | 60 seconds | +| `lockCheckDelay(Duration)` | Delay for checking lock availability | 30 seconds | +| `parallelism(int)` | Default parallelism for maintenance tasks | System default | +| `maxReadBack(int)` | Max snapshots to check during initialization | 100 | + +### Maintenance Task Common Options + +| Method | Description | Default Value | Type | +|--------|-------------|---------------|------| +| `scheduleOnCommitCount(int)` | Trigger after N commits | No automatic scheduling | int | +| `scheduleOnDataFileCount(int)` | Trigger after N data files | No automatic scheduling | int | +| `scheduleOnDataFileSize(long)` | Trigger after total data file size (bytes) | No automatic scheduling | long | +| `scheduleOnPosDeleteFileCount(int)` | Trigger after N positional delete files | No automatic scheduling | int | +| `scheduleOnPosDeleteRecordCount(long)` | Trigger after N positional delete records | No automatic scheduling | long | +| `scheduleOnEqDeleteFileCount(int)` | Trigger after N equality delete files | No automatic scheduling | int | +| `scheduleOnEqDeleteRecordCount(long)` | Trigger after N equality delete records | No automatic scheduling | long | +| `scheduleOnInterval(Duration)` | Trigger after time interval | No automatic scheduling | Duration | + +### ExpireSnapshots Configuration + +| Method | Description | Default Value | Type | +|--------|-------------|---------------|------| +| `maxSnapshotAge(Duration)` | Maximum age of snapshots to retain | 5 days | Duration | +| `retainLast(int)` | Minimum number of snapshots to retain | 1 | int | +| `deleteBatchSize(int)` | Number of files to delete in each batch | 1000 | int | +| `planningWorkerPoolSize(int)` | Number of worker threads for planning snapshot expiration | Shared worker pool | int | +| `cleanExpiredMetadata(boolean)` | Remove expired metadata files when expiring snapshots | false | boolean | + +### RewriteDataFiles Configuration + +| Method | Description | Default Value | Type | +|--------|-------------|---------------|------| +| `targetFileSizeBytes(long)` | Target size for rewritten files | Table property or 512MB | long | +| `minFileSizeBytes(long)` | Minimum size of files eligible for compaction | 0 | long | +| `maxFileSizeBytes(long)` | Maximum size of files eligible for compaction | Long.MAX_VALUE | long | +| `minInputFiles(int)` | Minimum number of files to trigger rewrite | 1 | int | +| `deleteFileThreshold(int)` | Minimum delete-file count per data file to force rewrite | Integer.MAX_VALUE | int | +| `rewriteAll(boolean)` | Rewrite all data files regardless of thresholds | false | boolean | +| `maxFileGroupSizeBytes(long)` | Maximum total size of a file group | 107374182400 (100GB) | long | +| `maxFilesToRewrite(int)` | Maximum number of files to rewrite per task | Integer.MAX_VALUE | int | +| `partialProgressEnabled(boolean)` | Enable partial progress commits | false | boolean | +| `partialProgressMaxCommits(int)` | Maximum commits allowed for partial progress when partialProgressEnabled is true | 10 | int | +| `maxRewriteBytes(long)` | Maximum bytes to rewrite per execution | Long.MAX_VALUE | long | + +## Complete Example + +```java +public class TableMaintenanceJob { + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.enableCheckpointing(60000); // Enable checkpointing + + // Configure table loader + TableLoader tableLoader = TableLoader.fromCatalog( + CatalogLoader.hive("my_catalog", configuration), + TableIdentifier.of("database", "table") + ); + + // Set up maintenance with comprehensive configuration + TableMaintenance.forTable(env, tableLoader, TriggerLockFactory.defaultLockFactory()) Review Comment: `TriggerLockFactory.defaultLockFactory()` We don't have this method. This part should be changed. -- 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