-
Notifications
You must be signed in to change notification settings - Fork 591
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
refactor(sink): impl SinkFormatter for AppendOnly and Upsert (#12321)
- Loading branch information
1 parent
f304ed2
commit 1877aed
Showing
8 changed files
with
274 additions
and
135 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
// Copyright 2023 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: KE, | ||
val_encoder: VE, | ||
} | ||
|
||
impl<KE, VE> AppendOnlyFormatter<KE, VE> { | ||
pub fn new(key_encoder: 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_generator(|| { | ||
for (op, row) in chunk.rows() { | ||
if op != Op::Insert { | ||
continue; | ||
} | ||
let event_key_object = Some(tri!(self.key_encoder.encode(row))); | ||
let event_object = Some(tri!(self.val_encoder.encode(row))); | ||
|
||
yield Ok((event_key_object, event_object)) | ||
} | ||
}) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,50 @@ | ||
// Copyright 2023 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::StreamChunk; | ||
|
||
use crate::sink::Result; | ||
|
||
mod append_only; | ||
mod upsert; | ||
|
||
pub use append_only::AppendOnlyFormatter; | ||
pub use upsert::UpsertFormatter; | ||
|
||
/// Transforms a `StreamChunk` into a sequence of key-value pairs according a specific format, | ||
/// for example append-only, upsert or debezium. | ||
pub trait SinkFormatter { | ||
type K; | ||
type V; | ||
|
||
fn format_chunk( | ||
&self, | ||
chunk: &StreamChunk, | ||
) -> impl Iterator<Item = Result<(Option<Self::K>, Option<Self::V>)>>; | ||
} | ||
|
||
/// `tri!` in generators yield `Err` and return `()` | ||
/// `?` in generators return `Err` | ||
#[macro_export] | ||
macro_rules! tri { | ||
($expr:expr) => { | ||
match $expr { | ||
Ok(val) => val, | ||
Err(err) => { | ||
yield Err(err); | ||
return; | ||
} | ||
} | ||
}; | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,61 @@ | ||
// Copyright 2023 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 UpsertFormatter<KE, VE> { | ||
key_encoder: KE, | ||
val_encoder: VE, | ||
} | ||
|
||
impl<KE, VE> UpsertFormatter<KE, VE> { | ||
pub fn new(key_encoder: KE, val_encoder: VE) -> Self { | ||
Self { | ||
key_encoder, | ||
val_encoder, | ||
} | ||
} | ||
} | ||
|
||
impl<KE: RowEncoder, VE: RowEncoder> SinkFormatter for UpsertFormatter<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_generator(|| { | ||
for (op, row) in chunk.rows() { | ||
let event_key_object = Some(tri!(self.key_encoder.encode(row))); | ||
|
||
let event_object = match op { | ||
Op::Insert | Op::UpdateInsert => Some(tri!(self.val_encoder.encode(row))), | ||
// Empty value with a key | ||
Op::Delete => None, | ||
Op::UpdateDelete => { | ||
// upsert semantic does not require update delete event | ||
continue; | ||
} | ||
}; | ||
|
||
yield Ok((event_key_object, event_object)) | ||
} | ||
}) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.