risingwave_connector/source/nats/enumerator/mod.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
// 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 std::sync::Arc;
use async_trait::async_trait;
use risingwave_common::bail;
use super::source::{NatsOffset, NatsSplit};
use super::NatsProperties;
use crate::error::ConnectorResult;
use crate::source::{SourceEnumeratorContextRef, SplitEnumerator, SplitId};
#[derive(Debug, Clone)]
pub struct NatsSplitEnumerator {
subject: String,
#[expect(dead_code)]
split_id: SplitId,
client: async_nats::Client,
}
#[async_trait]
impl SplitEnumerator for NatsSplitEnumerator {
type Properties = NatsProperties;
type Split = NatsSplit;
async fn new(
properties: Self::Properties,
_context: SourceEnumeratorContextRef,
) -> ConnectorResult<NatsSplitEnumerator> {
let client = properties.common.build_client().await?;
Ok(Self {
subject: properties.common.subject,
split_id: Arc::from("0"),
client,
})
}
async fn list_splits(&mut self) -> ConnectorResult<Vec<NatsSplit>> {
// Nats currently does not support list_splits API, if we simple return the default 0 without checking the client status, will result executor crash
let state = self.client.connection_state();
if state != async_nats::connection::State::Connected {
bail!(
"Nats connection status is not connected, current status is {:?}",
state
);
}
// TODO: to simplify the logic, return 1 split for first version
let nats_split = NatsSplit {
subject: self.subject.clone(),
split_id: Arc::from("0"), // be the same as `from_nats_jetstream_message`
start_sequence: NatsOffset::None,
};
Ok(vec![nats_split])
}
}