risingwave_connector/sink/
trivial.rs1use std::marker::PhantomData;
16
17use async_trait::async_trait;
18use risingwave_common::session_config::sink_decouple::SinkDecouple;
19
20use crate::sink::log_store::{LogStoreReadItem, TruncateOffset};
21use crate::sink::{
22 DummySinkCommitCoordinator, LogSinker, Result, Sink, SinkError, SinkLogReader, SinkParam,
23 SinkWriterParam,
24};
25
26pub const BLACKHOLE_SINK: &str = "blackhole";
27pub const TABLE_SINK: &str = "table";
28
29pub trait TrivialSinkName: Send + 'static {
30 const SINK_NAME: &'static str;
31}
32
33#[derive(Debug)]
34pub struct BlackHoleSinkName;
35
36impl TrivialSinkName for BlackHoleSinkName {
37 const SINK_NAME: &'static str = BLACKHOLE_SINK;
38}
39
40pub type BlackHoleSink = TrivialSink<BlackHoleSinkName>;
41
42#[derive(Debug)]
43pub struct TableSinkName;
44
45impl TrivialSinkName for TableSinkName {
46 const SINK_NAME: &'static str = TABLE_SINK;
47}
48
49pub type TableSink = TrivialSink<TableSinkName>;
50
51#[derive(Debug)]
52pub struct TrivialSink<T: TrivialSinkName>(PhantomData<T>);
53
54impl<T: TrivialSinkName> TryFrom<SinkParam> for TrivialSink<T> {
55 type Error = SinkError;
56
57 fn try_from(_value: SinkParam) -> std::result::Result<Self, Self::Error> {
58 Ok(Self(PhantomData))
59 }
60}
61
62impl<T: TrivialSinkName> Sink for TrivialSink<T> {
63 type Coordinator = DummySinkCommitCoordinator;
64 type LogSinker = Self;
65
66 const SINK_NAME: &'static str = T::SINK_NAME;
67
68 fn is_sink_decouple(_user_specified: &SinkDecouple) -> Result<bool> {
70 Ok(false)
71 }
72
73 async fn new_log_sinker(&self, _writer_env: SinkWriterParam) -> Result<Self::LogSinker> {
74 Ok(Self(PhantomData))
75 }
76
77 async fn validate(&self) -> Result<()> {
78 Ok(())
79 }
80}
81
82#[async_trait]
83impl<T: TrivialSinkName> LogSinker for TrivialSink<T> {
84 async fn consume_log_and_sink(self, mut log_reader: impl SinkLogReader) -> Result<!> {
85 log_reader.start_from(None).await?;
86 loop {
87 let (epoch, item) = log_reader.next_item().await?;
88 match item {
89 LogStoreReadItem::StreamChunk { chunk_id, .. } => {
90 log_reader.truncate(TruncateOffset::Chunk { epoch, chunk_id })?;
91 }
92 LogStoreReadItem::Barrier { .. } => {
93 log_reader.truncate(TruncateOffset::Barrier { epoch })?;
94 }
95 }
96 }
97 }
98}