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            if let Some(pk) = &config.primary_key {
104                let mut unique_column_ids = Vec::with_capacity(pk.len());
105                for col_name in pk {
106                    let id = param
107                        .columns
108                        .iter()
109                        .find(|col| col.name.as_str() == col_name)
110                        .ok_or_else(|| {
111                            SinkError::Config(anyhow!(
112                                "Primary key column {} not found in sink schema",
113                                col_name
114                            ))
115                        })?
116                        .column_id
117                        .get_id() as usize;
118                    unique_column_ids.push(id);
119                }
120                Some(unique_column_ids)
121            } else {
122                unreachable!()
123            }
124        } else {
125            None
126        };
127        Ok(Self {
128            config,
129            param,
130            unique_column_ids,
131        })
132    }
133}
134
135impl Sink for IcebergSink {
136    type LogSinker = CoordinatedLogSinker<IcebergSinkWriter>;
137
138    const SINK_NAME: &'static str = ICEBERG_SINK;
139
140    async fn validate(&self) -> Result<()> {
141        if "snowflake".eq_ignore_ascii_case(self.config.catalog_type()) {
142            bail!("Snowflake catalog only supports iceberg sources");
143        }
144
145        if "glue".eq_ignore_ascii_case(self.config.catalog_type()) {
146            risingwave_common::license::Feature::IcebergSinkWithGlue
147                .check_available()
148                .map_err(|e| anyhow::anyhow!(e))?;
149        }
150
151        // Enforce merge-on-read for append-only tables
152        IcebergConfig::validate_append_only_write_mode(
153            &self.config.r#type,
154            self.config.write_mode,
155        )?;
156
157        // Validate compaction type configuration
158        let compaction_type = self.config.compaction_type();
159
160        // Check COW mode constraints
161        // COW mode only supports 'full' compaction type
162        if self.config.write_mode == IcebergWriteMode::CopyOnWrite
163            && compaction_type != CompactionType::Full
164        {
165            bail!(
166                "'copy-on-write' mode only supports 'full' compaction type, got: '{}'",
167                compaction_type
168            );
169        }
170
171        match compaction_type {
172            CompactionType::SmallFiles => {
173                // 1. check license
174                risingwave_common::license::Feature::IcebergCompaction
175                    .check_available()
176                    .map_err(|e| anyhow::anyhow!(e))?;
177
178                // 2. check write mode
179                if self.config.write_mode != IcebergWriteMode::MergeOnRead {
180                    bail!(
181                        "'small-files' compaction type only supports 'merge-on-read' write mode, got: '{}'",
182                        self.config.write_mode
183                    );
184                }
185
186                // 3. check conflicting parameters
187                if self.config.delete_files_count_threshold.is_some() {
188                    bail!(
189                        "`compaction.delete-files-count-threshold` is not supported for 'small-files' compaction type"
190                    );
191                }
192            }
193            CompactionType::FilesWithDelete => {
194                // 1. check license
195                risingwave_common::license::Feature::IcebergCompaction
196                    .check_available()
197                    .map_err(|e| anyhow::anyhow!(e))?;
198
199                // 2. check write mode
200                if self.config.write_mode != IcebergWriteMode::MergeOnRead {
201                    bail!(
202                        "'files-with-delete' compaction type only supports 'merge-on-read' write mode, got: '{}'",
203                        self.config.write_mode
204                    );
205                }
206
207                // 3. check conflicting parameters
208                if self.config.small_files_threshold_mb.is_some() {
209                    bail!(
210                        "`compaction.small-files-threshold-mb` must not be set for 'files-with-delete' compaction type"
211                    );
212                }
213            }
214            CompactionType::Full => {
215                // Full compaction has no special requirements
216            }
217        }
218
219        let _ = self.create_and_validate_table().await?;
220        Ok(())
221    }
222
223    fn support_schema_change() -> bool {
224        true
225    }
226
227    fn validate_alter_config(config: &BTreeMap<String, String>) -> Result<()> {
228        let iceberg_config = IcebergConfig::from_btreemap(config.clone())?;
229
230        // Validate compaction interval
231        if let Some(compaction_interval) = iceberg_config.compaction_interval_sec {
232            if iceberg_config.enable_compaction && compaction_interval == 0 {
233                bail!(
234                    "`compaction-interval-sec` must be greater than 0 when `enable-compaction` is true"
235                );
236            }
237
238            tracing::info!(
239                "Alter config compaction_interval set to {} seconds",
240                compaction_interval
241            );
242        }
243
244        // Validate max snapshots
245        if let Some(max_snapshots) = iceberg_config.max_snapshots_num_before_compaction
246            && max_snapshots < 1
247        {
248            bail!(
249                "`compaction.max-snapshots-num` must be greater than 0, got: {}",
250                max_snapshots
251            );
252        }
253
254        // Validate target file size
255        if let Some(target_file_size_mb) = iceberg_config.target_file_size_mb
256            && target_file_size_mb == 0
257        {
258            bail!("`compaction.target_file_size_mb` must be greater than 0");
259        }
260
261        // Validate parquet max row group rows
262        if let Some(max_row_group_rows) = iceberg_config.write_parquet_max_row_group_rows
263            && max_row_group_rows == 0
264        {
265            bail!("`compaction.write_parquet_max_row_group_rows` must be greater than 0");
266        }
267
268        // Validate parquet compression codec
269        if let Some(ref compression) = iceberg_config.write_parquet_compression {
270            let valid_codecs = [
271                "uncompressed",
272                "snappy",
273                "gzip",
274                "lzo",
275                "brotli",
276                "lz4",
277                "zstd",
278            ];
279            if !valid_codecs.contains(&compression.to_lowercase().as_str()) {
280                bail!(
281                    "`compaction.write_parquet_compression` must be one of {:?}, got: {}",
282                    valid_codecs,
283                    compression
284                );
285            }
286        }
287
288        Ok(())
289    }
290
291    async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result<Self::LogSinker> {
292        let writer = IcebergSinkWriter::new(
293            self.config.clone(),
294            self.param.clone(),
295            writer_param.clone(),
296            self.unique_column_ids.clone(),
297        );
298
299        let commit_checkpoint_interval =
300            NonZeroU64::new(self.config.commit_checkpoint_interval).expect(
301                "commit_checkpoint_interval should be greater than 0, and it should be checked in config validation",
302            );
303        let log_sinker = CoordinatedLogSinker::new(
304            &writer_param,
305            self.param.clone(),
306            writer,
307            commit_checkpoint_interval,
308        )
309        .await?;
310
311        Ok(log_sinker)
312    }
313
314    fn is_coordinated_sink(&self) -> bool {
315        true
316    }
317
318    async fn new_coordinator(
319        &self,
320        iceberg_compact_stat_sender: Option<UnboundedSender<IcebergSinkCompactionUpdate>>,
321    ) -> Result<SinkCommitCoordinator> {
322        let catalog = self.config.create_catalog().await?;
323        let table = self.create_and_validate_table().await?;
324        let coordinator = IcebergSinkCommitter {
325            catalog,
326            table,
327            last_commit_epoch: 0,
328            sink_id: self.param.sink_id,
329            config: self.config.clone(),
330            param: self.param.clone(),
331            commit_retry_num: self.config.commit_retry_num,
332            iceberg_compact_stat_sender,
333        };
334        if self.config.is_exactly_once.unwrap_or_default() {
335            Ok(SinkCommitCoordinator::TwoPhase(Box::new(coordinator)))
336        } else {
337            Ok(SinkCommitCoordinator::SinglePhase(Box::new(coordinator)))
338        }
339    }
340}