risingwave_connector/sink/iceberg/prometheus/
monitored_partition_writer.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
// Copyright 2025 RisingWave Labs
//
// Licensed 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.

use iceberg::spec::DataFile;
use iceberg::writer::function_writer::fanout_partition_writer::{
    FanoutPartitionWriter, FanoutPartitionWriterBuilder,
};
use iceberg::writer::{IcebergWriter, IcebergWriterBuilder};
use iceberg::Result;
use risingwave_common::array::arrow::arrow_array_iceberg;
use risingwave_common::metrics::LabelGuardedIntGauge;

#[derive(Clone)]
pub struct MonitoredFanoutPartitionedWriterBuilder<B: IcebergWriterBuilder> {
    inner: FanoutPartitionWriterBuilder<B>,
    partition_num_metrics: LabelGuardedIntGauge<2>,
}

impl<B: IcebergWriterBuilder> MonitoredFanoutPartitionedWriterBuilder<B> {
    #[expect(dead_code)]
    pub fn new(
        inner: FanoutPartitionWriterBuilder<B>,
        partition_num: LabelGuardedIntGauge<2>,
    ) -> Self {
        Self {
            inner,
            partition_num_metrics: partition_num,
        }
    }
}

#[async_trait::async_trait]
impl<B: IcebergWriterBuilder> IcebergWriterBuilder for MonitoredFanoutPartitionedWriterBuilder<B> {
    type R = MonitoredFanoutPartitionedWriter<B>;

    async fn build(self) -> Result<Self::R> {
        let writer = self.inner.build().await?;
        Ok(MonitoredFanoutPartitionedWriter {
            inner: writer,
            partition_num_metrics: self.partition_num_metrics,
            last_partition_num: 0,
        })
    }
}

pub struct MonitoredFanoutPartitionedWriter<B: IcebergWriterBuilder> {
    inner: FanoutPartitionWriter<B>,
    partition_num_metrics: LabelGuardedIntGauge<2>,
    last_partition_num: usize,
}

impl<B: IcebergWriterBuilder> MonitoredFanoutPartitionedWriter<B> {
    pub fn update_metrics(&mut self) {
        let current_partition_num = self.inner.partition_num();
        let delta = current_partition_num as i64 - self.last_partition_num as i64;
        self.partition_num_metrics.add(delta);
        self.last_partition_num = current_partition_num;
    }
}

#[async_trait::async_trait]
impl<B: IcebergWriterBuilder> IcebergWriter for MonitoredFanoutPartitionedWriter<B> {
    async fn write(&mut self, batch: arrow_array_iceberg::RecordBatch) -> Result<()> {
        self.inner.write(batch).await?;
        self.update_metrics();
        Ok(())
    }

    async fn close(&mut self) -> Result<Vec<DataFile>> {
        self.update_metrics();
        let res = self.inner.close().await?;
        Ok(res)
    }
}