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