Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat(stream): support set nats consumer deliver policy as latest, earliest, by sequence, by timestamp #12176

Merged
merged 5 commits into from
Sep 14, 2023
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

15 changes: 12 additions & 3 deletions src/connector/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,9 @@ chrono = { version = "0.4", default-features = false, features = [
"clock",
"std",
] }
clickhouse = { git = "https://github.com/risingwavelabs/clickhouse.rs", rev = "622501c1c98c80baaf578c716d6903dde947804e", features = ["time"] }
clickhouse = { git = "https://github.com/risingwavelabs/clickhouse.rs", rev = "622501c1c98c80baaf578c716d6903dde947804e", features = [
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems unintended modifications to this file.

"time",
] }
csv = "1.2"
duration-str = "0.5.1"
enum-as-inner = "0.6"
Expand All @@ -56,8 +58,12 @@ itertools = "0.11"
jsonschema-transpiler = "1.10.0"
maplit = "1.0.2"
moka = { version = "0.11", features = ["future"] }
mysql_async = { version = "0.31", default-features = false, features = ["default"] }
mysql_common = { version = "0.29.2", default-features = false, features = ["chrono"] }
mysql_async = { version = "0.31", default-features = false, features = [
"default",
] }
mysql_common = { version = "0.29.2", default-features = false, features = [
"chrono",
] }
nexmark = { version = "0.2", features = ["serde"] }
num-bigint = "0.4"
opendal = "0.39"
Expand Down Expand Up @@ -109,6 +115,9 @@ tonic = { workspace = true }
tracing = "0.1"
url = "2"
urlencoding = "2"
time = "0.3.28"


[target.'cfg(not(madsim))'.dependencies]
workspace-hack = { path = "../workspace-hack" }

Expand Down
42 changes: 24 additions & 18 deletions src/connector/src/common.rs
Original file line number Diff line number Diff line change
Expand Up @@ -26,11 +26,12 @@ use risingwave_common::error::anyhow_error;
use serde_derive::{Deserialize, Serialize};
use serde_with::json::JsonString;
use serde_with::{serde_as, DisplayFromStr};
use time::OffsetDateTime;

use crate::aws_auth::AwsAuthProps;
use crate::deserialize_duration_from_string;
use crate::sink::SinkError;

use crate::source::nats::source::NatsOffset;
// The file describes the common abstractions for each connector and can be used in both source and
// sink.

Expand Down Expand Up @@ -395,7 +396,7 @@ impl NatsCommon {
pub(crate) async fn build_consumer(
&self,
split_id: i32,
start_sequence: Option<u64>,
start_sequence: NatsOffset,
) -> anyhow::Result<
async_nats::jetstream::consumer::Consumer<async_nats::jetstream::consumer::pull::Config>,
> {
Expand All @@ -406,23 +407,28 @@ impl NatsCommon {
ack_policy: jetstream::consumer::AckPolicy::None,
..Default::default()
};
match start_sequence {
Some(v) => {
let consumer = stream
.get_or_create_consumer(&name, {
config.deliver_policy = DeliverPolicy::ByStartSequence {
start_sequence: v + 1,
};
config
})
.await?;
Ok(consumer)
}
None => {
let consumer = stream.get_or_create_consumer(&name, config).await?;
Ok(consumer)

let deliver_policy = match start_sequence {
NatsOffset::Earliest => DeliverPolicy::All,
NatsOffset::Latest => DeliverPolicy::Last,
NatsOffset::SequenceNumber(v) => {
let parsed = v.parse::<u64>()?;
DeliverPolicy::ByStartSequence {
start_sequence: 1 + parsed,
Comment on lines +416 to +417
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the behavior is not consistent with other connectors, starts with seq should contain seqth message

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This one actually try to fix the problem that when resume.

If we use by sequence policy, this one may cause problem (we need to manuelly minus 1 for input sequence number). But currently I apply your suggestion and remove the by sequence policy. Then this would be fine.

}
}
}
NatsOffset::Timestamp(v) => DeliverPolicy::ByStartTime {
start_time: OffsetDateTime::from_unix_timestamp_nanos(v * 1_000_000)?,
},
NatsOffset::None => DeliverPolicy::All,
};
let consumer = stream
.get_or_create_consumer(&name, {
config.deliver_policy = deliver_policy;
config
})
.await?;
Ok(consumer)
}

pub(crate) async fn build_or_get_stream(
Expand Down
4 changes: 2 additions & 2 deletions src/connector/src/source/nats/enumerator/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@
use anyhow;
use async_trait::async_trait;

use super::source::NatsSplit;
use super::source::{NatsSplit, NatsOffset};
use super::NatsProperties;
use crate::source::{SourceEnumeratorContextRef, SplitEnumerator};

Expand Down Expand Up @@ -45,7 +45,7 @@ impl SplitEnumerator for NatsSplitEnumerator {
let nats_split = NatsSplit {
subject: self.subject.clone(),
split_num: 0, // be the same as `from_nats_jetstream_message`
start_sequence: None,
start_sequence: NatsOffset::None,
};

Ok(vec![nats_split])
Expand Down
15 changes: 15 additions & 0 deletions src/connector/src/source/nats/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,21 @@ pub const NATS_CONNECTOR: &str = "nats";
pub struct NatsProperties {
#[serde(flatten)]
pub common: NatsCommon,

#[serde(rename = "scan.startup.mode", alias = "nats.scan.startup.mode")]
pub scan_startup_mode: Option<String>,

#[serde(
rename = "scan.startup.sequence_number",
alias = "nats.scan.startup.sequence_number"
)]
pub start_sequence: Option<String>,

#[serde(
rename = "scan.startup.timestamp_millis",
alias = "nats.scan.startup.timestamp_millis"
)]
pub start_time: Option<String>,
}

impl NatsProperties {}
55 changes: 48 additions & 7 deletions src/connector/src/source/nats/source/reader.rs
Original file line number Diff line number Diff line change
Expand Up @@ -12,15 +12,15 @@
// See the License for the specific language governing permissions and
// limitations under the License.

use anyhow::Result;
use anyhow::{anyhow, Result};
use async_nats::jetstream::consumer;
use async_trait::async_trait;
use futures::StreamExt;
use futures_async_stream::try_stream;

use super::NatsOffset;
use crate::parser::ParserConfig;
use crate::source::common::{into_chunk_stream, CommonSplitReader};
use crate::source::nats::split::NatsSplit;
use crate::source::nats::NatsProperties;
use crate::source::{
BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SplitImpl, SplitReader,
Expand All @@ -31,6 +31,7 @@ pub struct NatsSplitReader {
properties: NatsProperties,
parser_config: ParserConfig,
source_ctx: SourceContextRef,
start_position: NatsOffset,
}

#[async_trait]
Expand All @@ -46,19 +47,59 @@ impl SplitReader for NatsSplitReader {
) -> Result<Self> {
// TODO: to simplify the logic, return 1 split for first version
assert!(splits.len() == 1);
let splits = splits
.into_iter()
.map(|split| split.into_nats().unwrap())
.collect::<Vec<NatsSplit>>();
let split = splits.into_iter().next().unwrap().into_nats().unwrap();
let start_position = match &split.start_sequence {
NatsOffset::None => match &properties.scan_startup_mode {
None => NatsOffset::Earliest,
Some(mode) => match mode.as_str() {
"latest" => NatsOffset::Latest,
"earliest" => NatsOffset::Earliest,
"timestamp_millis" => {
if let Some(time) = &properties.start_time {
NatsOffset::Timestamp(time.parse()?)
} else {
return Err(anyhow!("scan_startup_timestamp_millis is required"));
}
}
"sequence_number" => {
if let Some(seq) = &properties.start_sequence {
NatsOffset::SequenceNumber(seq.clone())
} else {
return Err(anyhow!("scan_startup_sequence_number is required"));
}
}
_ => {
return Err(anyhow!(
"invalid scan_startup_mode, accept earliest/latest/sequence_number/time"
))
}
},
},
start_position => start_position.to_owned(),
};

if !matches!(start_position, NatsOffset::SequenceNumber(_))
&& properties.start_sequence.is_some()
{
return Err(
anyhow!("scan.startup.mode need to be set to 'sequence_number' if you want to start with a specific sequence number")
);
}
if !matches!(start_position, NatsOffset::Timestamp(_)) && properties.start_time.is_some() {
return Err(
anyhow!("scan.startup.mode need to be set to 'timestamp_millis' if you want to start with a specific timestamp millis")
);
}
let consumer = properties
.common
.build_consumer(0, splits[0].start_sequence)
.build_consumer(0, start_position.clone())
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think it is a bad practice to hardcode the split_id 0 everywhere in the code. Here you should get the split id from the input split. And I suggest that you can save the split id to the SplitReader when you create the reader. Then you can use the split_id to build a SourceMessage instead of hardcoding it in the from_nats_jetstream_message.
Btw you may impl From<NatsMessage> for SourceMessage to do the conversion.

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for you remind. I already applied your suggestions. :)

.await?;
Ok(Self {
consumer,
properties,
parser_config,
source_ctx,
start_position,
})
}

Expand Down
20 changes: 17 additions & 3 deletions src/connector/src/source/nats/split.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,23 @@ use serde::{Deserialize, Serialize};

use crate::source::{SplitId, SplitMetaData};

#[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize, Hash)]
pub enum NatsOffset {
Earliest,
Latest,
SequenceNumber(String),
Timestamp(i128),
None,
}

/// The states of a NATS split, which will be persisted to checkpoint.
#[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)]
pub struct NatsSplit {
pub(crate) subject: String,
// TODO: to simplify the logic, return 1 split for first version. May use parallelism in
// future.
pub(crate) split_num: i32,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

may refactor to split_id

pub(crate) start_sequence: Option<u64>,
pub(crate) start_sequence: NatsOffset,
}

impl SplitMetaData for NatsSplit {
Expand All @@ -44,7 +53,7 @@ impl SplitMetaData for NatsSplit {
}

impl NatsSplit {
pub fn new(subject: String, split_num: i32, start_sequence: Option<u64>) -> Self {
pub fn new(subject: String, split_num: i32, start_sequence: NatsOffset) -> Self {
Self {
subject,
split_num,
Expand All @@ -53,7 +62,12 @@ impl NatsSplit {
}

pub fn update_with_offset(&mut self, start_sequence: String) -> anyhow::Result<()> {
self.start_sequence = Some(start_sequence.as_str().parse::<u64>().unwrap());
let start_sequence = if start_sequence.is_empty() {
NatsOffset::Earliest
} else {
NatsOffset::SequenceNumber(start_sequence)
};
self.start_sequence = start_sequence;
Ok(())
}
}