risingwave_connector/sink/formatter/append_only.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
// Copyright 2024 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 risingwave_common::array::Op;
use super::{Result, SinkFormatter, StreamChunk};
use crate::sink::encoder::RowEncoder;
use crate::tri;
pub struct AppendOnlyFormatter<KE, VE> {
key_encoder: Option<KE>,
val_encoder: VE,
}
impl<KE, VE> AppendOnlyFormatter<KE, VE> {
pub fn new(key_encoder: Option<KE>, val_encoder: VE) -> Self {
Self {
key_encoder,
val_encoder,
}
}
}
impl<KE: RowEncoder, VE: RowEncoder> SinkFormatter for AppendOnlyFormatter<KE, VE> {
type K = KE::Output;
type V = VE::Output;
fn format_chunk(
&self,
chunk: &StreamChunk,
) -> impl Iterator<Item = Result<(Option<Self::K>, Option<Self::V>)>> {
std::iter::from_coroutine(
#[coroutine]
|| {
for (op, row) in chunk.rows() {
if op != Op::Insert {
continue;
}
let event_key_object = match &self.key_encoder {
Some(key_encoder) => Some(tri!(key_encoder.encode(row))),
None => None,
};
let event_object = Some(tri!(self.val_encoder.encode(row)));
yield Ok((event_key_object, event_object))
}
},
)
}
}