risingwave_connector/sink/iceberg/
mod.rs

1// Copyright 2023 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15#[cfg(test)]
16mod test;
17
18mod commit;
19mod config;
20mod create_table;
21mod prometheus;
22mod writer;
23
24use std::collections::BTreeMap;
25use std::fmt::Debug;
26use std::num::NonZeroU64;
27
28use anyhow::{Context, anyhow};
29pub use commit::*;
30pub use config::*;
31pub use create_table::*;
32use iceberg::table::Table;
33use risingwave_common::bail;
34use tokio::sync::mpsc::UnboundedSender;
35pub use writer::*;
36
37use super::{
38    GLOBAL_SINK_METRICS, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, Sink,
39    SinkError, SinkWriterParam,
40};
41use crate::connector_common::IcebergSinkCompactionUpdate;
42use crate::enforce_secret::EnforceSecret;
43use crate::sink::coordinate::CoordinatedLogSinker;
44use crate::sink::{Result, SinkCommitCoordinator, SinkParam};
45
46pub const ICEBERG_SINK: &str = "iceberg";
47
48pub struct IcebergSink {
49    pub config: IcebergConfig,
50    param: SinkParam,
51    // In upsert mode, it never be None and empty.
52    unique_column_ids: Option<Vec<usize>>,
53}
54
55impl EnforceSecret for IcebergSink {
56    fn enforce_secret<'a>(
57        prop_iter: impl Iterator<Item = &'a str>,
58    ) -> crate::error::ConnectorResult<()> {
59        for prop in prop_iter {
60            IcebergConfig::enforce_one(prop)?;
61        }
62        Ok(())
63    }
64}
65
66impl TryFrom<SinkParam> for IcebergSink {
67    type Error = SinkError;
68
69    fn try_from(param: SinkParam) -> std::result::Result<Self, Self::Error> {
70        let config = IcebergConfig::from_btreemap(param.properties.clone())?;
71        IcebergSink::new(config, param)
72    }
73}
74
75impl Debug for IcebergSink {
76    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
77        f.debug_struct("IcebergSink")
78            .field("config", &self.config)
79            .finish()
80    }
81}
82
83impl IcebergSink {
84    pub async fn create_and_validate_table(&self) -> Result<Table> {
85        create_and_validate_table_impl(&self.config, &self.param).await
86    }
87
88    pub async fn create_table_if_not_exists(&self) -> Result<()> {
89        create_table_if_not_exists_impl(&self.config, &self.param).await
90    }
91
92    pub fn new(config: IcebergConfig, param: SinkParam) -> Result<Self> {
93        if let Some(order_key) = &config.order_key {
94            validate_order_key_columns(
95                order_key,
96                param.columns.iter().map(|column| column.name.as_str()),
97            )
98            .context("invalid order_key")
99            .map_err(SinkError::Config)?;
100        }
101
102        let unique_column_ids = if config.r#type == SINK_TYPE_UPSERT && !config.force_append_only {
103            // Use the pk indices resolved by the frontend instead of re-matching by name.
104            let pk_indices = param
105                .downstream_pk
106                .as_ref()
107                .filter(|pk| !pk.is_empty())
108                .ok_or_else(|| {
109                    SinkError::Config(anyhow!(
110                        "primary key must be specified for upsert iceberg sink"
111                    ))
112                })?;
113            let unique_column_ids = pk_indices
114                .iter()
115                .map(|&idx| param.columns[idx].column_id.get_id() as usize)
116                .collect();
117            Some(unique_column_ids)
118        } else {
119            None
120        };
121        Ok(Self {
122            config,
123            param,
124            unique_column_ids,
125        })
126    }
127}
128
129impl Sink for IcebergSink {
130    type LogSinker = CoordinatedLogSinker<IcebergSinkWriter>;
131
132    const SINK_NAME: &'static str = ICEBERG_SINK;
133
134    async fn validate(&self) -> Result<()> {
135        if "snowflake".eq_ignore_ascii_case(self.config.catalog_type()) {
136            bail!("Snowflake catalog only supports iceberg sources");
137        }
138
139        if "glue".eq_ignore_ascii_case(self.config.catalog_type()) {
140            risingwave_common::license::Feature::IcebergSinkWithGlue
141                .check_available()
142                .map_err(|e| anyhow::anyhow!(e))?;
143        }
144
145        // Enforce merge-on-read for append-only tables
146        IcebergConfig::validate_append_only_write_mode(
147            &self.config.r#type,
148            self.config.write_mode,
149        )?;
150
151        // Validate compaction type configuration
152        let compaction_type = self.config.compaction_type();
153
154        // Check COW mode constraints
155        // COW mode only supports 'full' compaction type
156        if self.config.write_mode == IcebergWriteMode::CopyOnWrite
157            && compaction_type != CompactionType::Full
158        {
159            bail!(
160                "'copy-on-write' mode only supports 'full' compaction type, got: '{}'",
161                compaction_type
162            );
163        }
164
165        match compaction_type {
166            CompactionType::SmallFiles => {
167                // 1. check license
168                risingwave_common::license::Feature::IcebergCompaction
169                    .check_available()
170                    .map_err(|e| anyhow::anyhow!(e))?;
171
172                // 2. check write mode
173                if self.config.write_mode != IcebergWriteMode::MergeOnRead {
174                    bail!(
175                        "'small-files' compaction type only supports 'merge-on-read' write mode, got: '{}'",
176                        self.config.write_mode
177                    );
178                }
179
180                // 3. check conflicting parameters
181                if self.config.delete_files_count_threshold.is_some() {
182                    bail!(
183                        "`compaction.delete-files-count-threshold` is not supported for 'small-files' compaction type"
184                    );
185                }
186            }
187            CompactionType::FilesWithDelete => {
188                // 1. check license
189                risingwave_common::license::Feature::IcebergCompaction
190                    .check_available()
191                    .map_err(|e| anyhow::anyhow!(e))?;
192
193                // 2. check write mode
194                if self.config.write_mode != IcebergWriteMode::MergeOnRead {
195                    bail!(
196                        "'files-with-delete' compaction type only supports 'merge-on-read' write mode, got: '{}'",
197                        self.config.write_mode
198                    );
199                }
200
201                // 3. check conflicting parameters
202                if self.config.small_files_threshold_mb.is_some() {
203                    bail!(
204                        "`compaction.small-files-threshold-mb` must not be set for 'files-with-delete' compaction type"
205                    );
206                }
207            }
208            CompactionType::Full => {
209                // Full compaction has no special requirements
210            }
211        }
212
213        let _ = self.create_and_validate_table().await?;
214        Ok(())
215    }
216
217    fn support_schema_change() -> bool {
218        true
219    }
220
221    fn validate_alter_config(config: &BTreeMap<String, String>) -> Result<()> {
222        let iceberg_config = IcebergConfig::from_btreemap(config.clone())?;
223
224        // Validate compaction interval
225        if let Some(compaction_interval) = iceberg_config.compaction_interval_sec {
226            if iceberg_config.enable_compaction && compaction_interval == 0 {
227                bail!(
228                    "`compaction-interval-sec` must be greater than 0 when `enable-compaction` is true"
229                );
230            }
231
232            tracing::info!(
233                "Alter config compaction_interval set to {} seconds",
234                compaction_interval
235            );
236        }
237
238        // Validate max snapshots
239        if let Some(max_snapshots) = iceberg_config.max_snapshots_num_before_compaction
240            && max_snapshots < 1
241        {
242            bail!(
243                "`compaction.max-snapshots-num` must be greater than 0, got: {}",
244                max_snapshots
245            );
246        }
247
248        // Validate target file size
249        if let Some(target_file_size_mb) = iceberg_config.target_file_size_mb
250            && target_file_size_mb == 0
251        {
252            bail!("`compaction.target_file_size_mb` must be greater than 0");
253        }
254
255        // Validate parquet max row group rows
256        if let Some(max_row_group_rows) = iceberg_config.write_parquet_max_row_group_rows
257            && max_row_group_rows == 0
258        {
259            bail!("`compaction.write_parquet_max_row_group_rows` must be greater than 0");
260        }
261
262        // Validate parquet max row group bytes
263        if let Some(max_row_group_bytes) = iceberg_config.write_parquet_max_row_group_bytes
264            && max_row_group_bytes == 0
265        {
266            bail!("`compaction.write_parquet_max_row_group_bytes` must be greater than 0");
267        }
268
269        // Validate parquet compression codec
270        if let Some(ref compression) = iceberg_config.write_parquet_compression {
271            let valid_codecs = [
272                "uncompressed",
273                "snappy",
274                "gzip",
275                "lzo",
276                "brotli",
277                "lz4",
278                "zstd",
279            ];
280            if !valid_codecs.contains(&compression.to_lowercase().as_str()) {
281                bail!(
282                    "`compaction.write_parquet_compression` must be one of {:?}, got: {}",
283                    valid_codecs,
284                    compression
285                );
286            }
287        }
288
289        Ok(())
290    }
291
292    async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result<Self::LogSinker> {
293        let writer = IcebergSinkWriter::new(
294            self.config.clone(),
295            self.param.clone(),
296            writer_param.clone(),
297            self.unique_column_ids.clone(),
298        );
299
300        let commit_checkpoint_interval =
301            NonZeroU64::new(self.config.commit_checkpoint_interval).expect(
302                "commit_checkpoint_interval should be greater than 0, and it should be checked in config validation",
303            );
304        let log_sinker = CoordinatedLogSinker::new(
305            &writer_param,
306            self.param.clone(),
307            writer,
308            commit_checkpoint_interval,
309        )
310        .await?;
311
312        Ok(log_sinker)
313    }
314
315    fn is_coordinated_sink(&self) -> bool {
316        true
317    }
318
319    async fn new_coordinator(
320        &self,
321        iceberg_compact_stat_sender: Option<UnboundedSender<IcebergSinkCompactionUpdate>>,
322    ) -> Result<SinkCommitCoordinator> {
323        let catalog = self.config.create_catalog().await?;
324        let table = self.create_and_validate_table().await?;
325        let coordinator = IcebergSinkCommitter {
326            catalog,
327            table,
328            last_commit_epoch: 0,
329            sink_id: self.param.sink_id,
330            config: self.config.clone(),
331            param: self.param.clone(),
332            commit_retry_num: self.config.commit_retry_num,
333            iceberg_compact_stat_sender,
334        };
335        if self.config.is_exactly_once.unwrap_or_default() {
336            Ok(SinkCommitCoordinator::TwoPhase(Box::new(coordinator)))
337        } else {
338            Ok(SinkCommitCoordinator::SinglePhase(Box::new(coordinator)))
339        }
340    }
341}