@@ -4,11 +4,18 @@ version.workspace = true
|
||||
edition.workspace = true
|
||||
|
||||
[dependencies]
|
||||
sq-capnp-interface = { workspace = true }
|
||||
sq-grpc-interface = { workspace = true }
|
||||
sq-models = { workspace = true }
|
||||
|
||||
capnp = { workspace = true }
|
||||
bytes = { workspace = true }
|
||||
|
||||
anyhow = { workspace = true }
|
||||
tokio = { workspace = true }
|
||||
tracing = { workspace = true }
|
||||
tonic = { workspace = true }
|
||||
thiserror = { workspace = true }
|
||||
tokio-stream = { workspace = true }
|
||||
tokio-util = { workspace = true, features = ["codec"] }
|
||||
futures = { workspace = true }
|
||||
|
||||
172
crates/sq-sdk/src/batch_producer.rs
Normal file
172
crates/sq-sdk/src/batch_producer.rs
Normal file
@@ -0,0 +1,172 @@
|
||||
use std::time::Duration;
|
||||
|
||||
use sq_grpc_interface::AckMode;
|
||||
use tokio::sync::{mpsc, oneshot};
|
||||
use tokio::time::MissedTickBehavior;
|
||||
|
||||
use crate::error::SqError;
|
||||
use crate::producer::{GrpcProducer, GrpcProducerConfig, ProducerMessage, SendResult};
|
||||
|
||||
/// Configuration for a gRPC batching producer.
|
||||
pub struct GrpcBatchProducerConfig {
|
||||
/// Server address (e.g., "http://127.0.0.1:6060").
|
||||
pub address: String,
|
||||
/// Default ack mode for publish requests.
|
||||
pub ack_mode: AckMode,
|
||||
/// Producer identifier.
|
||||
pub producer_id: String,
|
||||
/// Maximum messages to accumulate before flushing (default: 1000).
|
||||
pub max_batch_size: usize,
|
||||
/// Flush interval in milliseconds (default: 10).
|
||||
pub flush_interval_ms: u64,
|
||||
/// Backpressure channel capacity (default: 10_000).
|
||||
pub channel_capacity: usize,
|
||||
}
|
||||
|
||||
impl Default for GrpcBatchProducerConfig {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
address: "http://127.0.0.1:6060".to_string(),
|
||||
ack_mode: AckMode::All,
|
||||
producer_id: "default".to_string(),
|
||||
max_batch_size: 1000,
|
||||
flush_interval_ms: 10,
|
||||
channel_capacity: 10_000,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
struct BatchRequest {
|
||||
message: ProducerMessage,
|
||||
reply: oneshot::Sender<Result<SendResult, SqError>>,
|
||||
}
|
||||
|
||||
/// A gRPC batching producer that accumulates messages and flushes them in batches.
|
||||
///
|
||||
/// Messages are queued immediately via `send()` (non-blocking enqueue) and
|
||||
/// flushed to the server either when the batch reaches `max_batch_size` or
|
||||
/// when the `flush_interval` timer fires — whichever comes first.
|
||||
///
|
||||
/// `send()` takes `&self`, so `GrpcBatchProducer` can be shared via `Arc` across tasks.
|
||||
pub struct GrpcBatchProducer {
|
||||
tx: mpsc::Sender<BatchRequest>,
|
||||
_flush_task: tokio::task::JoinHandle<()>,
|
||||
}
|
||||
|
||||
impl GrpcBatchProducer {
|
||||
/// Connect to an SQ server and create a batching producer.
|
||||
pub async fn connect(config: GrpcBatchProducerConfig) -> Result<Self, SqError> {
|
||||
let producer = GrpcProducer::connect(GrpcProducerConfig {
|
||||
address: config.address,
|
||||
ack_mode: config.ack_mode,
|
||||
producer_id: config.producer_id,
|
||||
})
|
||||
.await?;
|
||||
|
||||
let (tx, rx) = mpsc::channel(config.channel_capacity);
|
||||
|
||||
let flush_task = tokio::spawn(flush_loop(
|
||||
rx,
|
||||
producer,
|
||||
config.max_batch_size,
|
||||
Duration::from_millis(config.flush_interval_ms),
|
||||
));
|
||||
|
||||
Ok(Self {
|
||||
tx,
|
||||
_flush_task: flush_task,
|
||||
})
|
||||
}
|
||||
|
||||
/// Queue a message for batched sending. Returns the result once the batch
|
||||
/// containing this message has been flushed and acknowledged by the server.
|
||||
pub async fn send(&self, message: ProducerMessage) -> Result<SendResult, SqError> {
|
||||
let (reply_tx, reply_rx) = oneshot::channel();
|
||||
self.tx
|
||||
.send(BatchRequest {
|
||||
message,
|
||||
reply: reply_tx,
|
||||
})
|
||||
.await
|
||||
.map_err(|_| SqError::Connection("batch producer closed".to_string()))?;
|
||||
|
||||
reply_rx
|
||||
.await
|
||||
.map_err(|_| SqError::Connection("batch producer flush task dropped".to_string()))?
|
||||
}
|
||||
|
||||
/// Shut down the producer, flushing any remaining messages.
|
||||
///
|
||||
/// Dropping the `BatchProducer` also triggers a flush of pending messages,
|
||||
/// but `close()` lets you await completion.
|
||||
pub async fn close(self) {
|
||||
drop(self.tx);
|
||||
let _ = self._flush_task.await;
|
||||
}
|
||||
}
|
||||
|
||||
async fn flush_loop(
|
||||
mut rx: mpsc::Receiver<BatchRequest>,
|
||||
mut producer: GrpcProducer,
|
||||
max_batch_size: usize,
|
||||
flush_interval: Duration,
|
||||
) {
|
||||
let mut pending: Vec<BatchRequest> = Vec::with_capacity(max_batch_size);
|
||||
let mut interval = tokio::time::interval(flush_interval);
|
||||
interval.set_missed_tick_behavior(MissedTickBehavior::Delay);
|
||||
// Consume the first immediate tick.
|
||||
interval.tick().await;
|
||||
|
||||
loop {
|
||||
let should_flush = tokio::select! {
|
||||
msg = rx.recv() => match msg {
|
||||
Some(req) => {
|
||||
pending.push(req);
|
||||
pending.len() >= max_batch_size
|
||||
}
|
||||
None => {
|
||||
// Channel closed — flush remaining and exit.
|
||||
if !pending.is_empty() {
|
||||
flush(&mut producer, &mut pending).await;
|
||||
}
|
||||
return;
|
||||
}
|
||||
},
|
||||
_ = interval.tick() => !pending.is_empty(),
|
||||
};
|
||||
|
||||
if should_flush {
|
||||
flush(&mut producer, &mut pending).await;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn flush(producer: &mut GrpcProducer, pending: &mut Vec<BatchRequest>) {
|
||||
let batch: Vec<BatchRequest> = std::mem::take(pending);
|
||||
|
||||
let messages: Vec<ProducerMessage> = batch
|
||||
.iter()
|
||||
.map(|req| ProducerMessage {
|
||||
topic: req.message.topic.clone(),
|
||||
key: req.message.key.clone(),
|
||||
value: req.message.value.clone(),
|
||||
headers: req.message.headers.clone(),
|
||||
})
|
||||
.collect();
|
||||
|
||||
match producer.send_batch(messages).await {
|
||||
Ok(results) => {
|
||||
for (req, result) in batch.into_iter().zip(results) {
|
||||
let _ = req.reply.send(Ok(result));
|
||||
}
|
||||
}
|
||||
Err(e) => {
|
||||
let msg = e.to_string();
|
||||
for req in batch {
|
||||
let _ = req
|
||||
.reply
|
||||
.send(Err(SqError::Server(msg.clone())));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
169
crates/sq-sdk/src/capnp_batch_producer.rs
Normal file
169
crates/sq-sdk/src/capnp_batch_producer.rs
Normal file
@@ -0,0 +1,169 @@
|
||||
use std::time::Duration;
|
||||
|
||||
use tokio::sync::{mpsc, oneshot};
|
||||
use tokio::time::MissedTickBehavior;
|
||||
|
||||
use crate::capnp_producer::{Producer, ProducerConfig};
|
||||
use crate::error::SqError;
|
||||
use crate::producer::{ProducerMessage, SendResult};
|
||||
use crate::types::AckMode;
|
||||
|
||||
/// Configuration for a batching producer (Cap'n Proto transport).
|
||||
pub struct BatchProducerConfig {
|
||||
/// Server address (e.g., "127.0.0.1:6064").
|
||||
pub address: String,
|
||||
/// Default ack mode for publish requests.
|
||||
pub ack_mode: AckMode,
|
||||
/// Producer identifier.
|
||||
pub producer_id: String,
|
||||
/// Maximum messages to accumulate before flushing (default: 1000).
|
||||
pub max_batch_size: usize,
|
||||
/// Flush interval in milliseconds (default: 10).
|
||||
pub flush_interval_ms: u64,
|
||||
/// Backpressure channel capacity (default: 10_000).
|
||||
pub channel_capacity: usize,
|
||||
}
|
||||
|
||||
impl Default for BatchProducerConfig {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
address: "127.0.0.1:6064".to_string(),
|
||||
ack_mode: AckMode::All,
|
||||
producer_id: "default".to_string(),
|
||||
max_batch_size: 1000,
|
||||
flush_interval_ms: 10,
|
||||
channel_capacity: 10_000,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
struct BatchRequest {
|
||||
message: ProducerMessage,
|
||||
reply: oneshot::Sender<Result<SendResult, SqError>>,
|
||||
}
|
||||
|
||||
/// A batching producer that accumulates messages and flushes them in batches
|
||||
/// over the Cap'n Proto transport.
|
||||
///
|
||||
/// Messages are queued immediately via `send()` (non-blocking enqueue) and
|
||||
/// flushed to the server either when the batch reaches `max_batch_size` or
|
||||
/// when the `flush_interval` timer fires — whichever comes first.
|
||||
///
|
||||
/// `send()` takes `&self`, so `BatchProducer` can be shared via `Arc` across tasks.
|
||||
pub struct BatchProducer {
|
||||
tx: mpsc::Sender<BatchRequest>,
|
||||
_flush_task: tokio::task::JoinHandle<()>,
|
||||
}
|
||||
|
||||
impl BatchProducer {
|
||||
/// Connect to an SQ server and create a batching producer.
|
||||
pub async fn connect(config: BatchProducerConfig) -> Result<Self, SqError> {
|
||||
let producer = Producer::connect(ProducerConfig {
|
||||
address: config.address,
|
||||
ack_mode: config.ack_mode,
|
||||
producer_id: config.producer_id,
|
||||
})
|
||||
.await?;
|
||||
|
||||
let (tx, rx) = mpsc::channel(config.channel_capacity);
|
||||
|
||||
let flush_task = tokio::spawn(flush_loop(
|
||||
rx,
|
||||
producer,
|
||||
config.max_batch_size,
|
||||
Duration::from_millis(config.flush_interval_ms),
|
||||
));
|
||||
|
||||
Ok(Self {
|
||||
tx,
|
||||
_flush_task: flush_task,
|
||||
})
|
||||
}
|
||||
|
||||
/// Queue a message for batched sending. Returns the result once the batch
|
||||
/// containing this message has been flushed and acknowledged by the server.
|
||||
pub async fn send(&self, message: ProducerMessage) -> Result<SendResult, SqError> {
|
||||
let (reply_tx, reply_rx) = oneshot::channel();
|
||||
self.tx
|
||||
.send(BatchRequest {
|
||||
message,
|
||||
reply: reply_tx,
|
||||
})
|
||||
.await
|
||||
.map_err(|_| SqError::Connection("batch producer closed".to_string()))?;
|
||||
|
||||
reply_rx
|
||||
.await
|
||||
.map_err(|_| SqError::Connection("batch producer flush task dropped".to_string()))?
|
||||
}
|
||||
|
||||
/// Shut down the producer, flushing any remaining messages.
|
||||
pub async fn close(self) {
|
||||
drop(self.tx);
|
||||
let _ = self._flush_task.await;
|
||||
}
|
||||
}
|
||||
|
||||
async fn flush_loop(
|
||||
mut rx: mpsc::Receiver<BatchRequest>,
|
||||
mut producer: Producer,
|
||||
max_batch_size: usize,
|
||||
flush_interval: Duration,
|
||||
) {
|
||||
let mut pending: Vec<BatchRequest> = Vec::with_capacity(max_batch_size);
|
||||
let mut interval = tokio::time::interval(flush_interval);
|
||||
interval.set_missed_tick_behavior(MissedTickBehavior::Delay);
|
||||
// Consume the first immediate tick.
|
||||
interval.tick().await;
|
||||
|
||||
loop {
|
||||
let should_flush = tokio::select! {
|
||||
msg = rx.recv() => match msg {
|
||||
Some(req) => {
|
||||
pending.push(req);
|
||||
pending.len() >= max_batch_size
|
||||
}
|
||||
None => {
|
||||
// Channel closed — flush remaining and exit.
|
||||
if !pending.is_empty() {
|
||||
flush(&mut producer, &mut pending).await;
|
||||
}
|
||||
return;
|
||||
}
|
||||
},
|
||||
_ = interval.tick() => !pending.is_empty(),
|
||||
};
|
||||
|
||||
if should_flush {
|
||||
flush(&mut producer, &mut pending).await;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
async fn flush(producer: &mut Producer, pending: &mut Vec<BatchRequest>) {
|
||||
let batch: Vec<BatchRequest> = std::mem::take(pending);
|
||||
|
||||
let messages: Vec<ProducerMessage> = batch
|
||||
.iter()
|
||||
.map(|req| ProducerMessage {
|
||||
topic: req.message.topic.clone(),
|
||||
key: req.message.key.clone(),
|
||||
value: req.message.value.clone(),
|
||||
headers: req.message.headers.clone(),
|
||||
})
|
||||
.collect();
|
||||
|
||||
match producer.send_batch(messages).await {
|
||||
Ok(results) => {
|
||||
for (req, result) in batch.into_iter().zip(results) {
|
||||
let _ = req.reply.send(Ok(result));
|
||||
}
|
||||
}
|
||||
Err(e) => {
|
||||
let msg = e.to_string();
|
||||
for req in batch {
|
||||
let _ = req.reply.send(Err(SqError::Server(msg.clone())));
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
46
crates/sq-sdk/src/capnp_connection.rs
Normal file
46
crates/sq-sdk/src/capnp_connection.rs
Normal file
@@ -0,0 +1,46 @@
|
||||
use futures::SinkExt;
|
||||
use sq_capnp_interface::codec::{Frame, SqCodec};
|
||||
use tokio::net::TcpStream;
|
||||
use tokio_stream::StreamExt;
|
||||
use tokio_util::codec::Framed;
|
||||
|
||||
use crate::error::SqError;
|
||||
|
||||
/// A TCP connection with Cap'n Proto framing.
|
||||
pub struct Connection {
|
||||
framed: Framed<TcpStream, SqCodec>,
|
||||
}
|
||||
|
||||
impl Connection {
|
||||
/// Connect to an SQ server's capnp data plane.
|
||||
/// Address should be "host:port" (e.g., "127.0.0.1:6064").
|
||||
pub async fn connect(address: &str) -> Result<Self, SqError> {
|
||||
let stream = TcpStream::connect(address)
|
||||
.await
|
||||
.map_err(|e| SqError::Connection(e.to_string()))?;
|
||||
stream
|
||||
.set_nodelay(true)
|
||||
.map_err(|e| SqError::Connection(e.to_string()))?;
|
||||
|
||||
Ok(Self {
|
||||
framed: Framed::new(stream, SqCodec::new()),
|
||||
})
|
||||
}
|
||||
|
||||
/// Send a frame over the connection.
|
||||
pub async fn send_frame(&mut self, frame: Frame) -> Result<(), SqError> {
|
||||
self.framed
|
||||
.send(frame)
|
||||
.await
|
||||
.map_err(|e| SqError::Connection(e.to_string()))
|
||||
}
|
||||
|
||||
/// Receive the next frame from the connection.
|
||||
pub async fn recv_frame(&mut self) -> Result<Frame, SqError> {
|
||||
match self.framed.next().await {
|
||||
Some(Ok(frame)) => Ok(frame),
|
||||
Some(Err(e)) => Err(SqError::Connection(e.to_string())),
|
||||
None => Err(SqError::Connection("connection closed".to_string())),
|
||||
}
|
||||
}
|
||||
}
|
||||
224
crates/sq-sdk/src/capnp_consumer.rs
Normal file
224
crates/sq-sdk/src/capnp_consumer.rs
Normal file
@@ -0,0 +1,224 @@
|
||||
use sq_capnp_interface::codec::{self, OP_ERROR, OP_SUBSCRIBE_END, OP_SUBSCRIBE_REQ, OP_SUBSCRIBE_RES, OP_ACK_REQ, OP_ACK_RES};
|
||||
use sq_capnp_interface::data_plane_capnp;
|
||||
|
||||
use crate::capnp_connection::Connection;
|
||||
use crate::consumer::ReceivedMessage;
|
||||
use crate::error::SqError;
|
||||
|
||||
/// Configuration for an SQ consumer (Cap'n Proto transport).
|
||||
pub struct ConsumerConfig {
|
||||
/// Server address (e.g., "127.0.0.1:6064").
|
||||
pub address: String,
|
||||
/// Consumer group name.
|
||||
pub consumer_group: String,
|
||||
/// Topic to consume from.
|
||||
pub topic: String,
|
||||
/// Partition to consume from.
|
||||
pub partition: u32,
|
||||
/// Whether to automatically commit offsets.
|
||||
pub auto_commit: bool,
|
||||
/// Maximum number of messages per batch.
|
||||
pub max_poll_records: u32,
|
||||
/// Optional start offset (overrides consumer group committed offset).
|
||||
pub start_offset: Option<u64>,
|
||||
}
|
||||
|
||||
impl Default for ConsumerConfig {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
address: "127.0.0.1:6064".to_string(),
|
||||
consumer_group: "default".to_string(),
|
||||
topic: String::new(),
|
||||
partition: 0,
|
||||
auto_commit: true,
|
||||
max_poll_records: 100,
|
||||
start_offset: None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// SQ consumer using Cap'n Proto over TCP.
|
||||
/// Uses two connections: one for subscribe streaming, one for ack requests.
|
||||
pub struct Consumer {
|
||||
subscribe_conn: Connection,
|
||||
ack_conn: Connection,
|
||||
config: ConsumerConfig,
|
||||
stream_started: bool,
|
||||
last_offset: Option<u64>,
|
||||
}
|
||||
|
||||
impl Consumer {
|
||||
/// Connect to an SQ server and create a new consumer.
|
||||
pub async fn connect(config: ConsumerConfig) -> Result<Self, SqError> {
|
||||
let subscribe_conn = Connection::connect(&config.address).await?;
|
||||
let ack_conn = Connection::connect(&config.address).await?;
|
||||
|
||||
Ok(Self {
|
||||
subscribe_conn,
|
||||
ack_conn,
|
||||
config,
|
||||
stream_started: false,
|
||||
last_offset: None,
|
||||
})
|
||||
}
|
||||
|
||||
/// Poll for new messages.
|
||||
/// On first call, sends the SubscribeRequest. Subsequent calls read response frames.
|
||||
pub async fn poll(&mut self) -> Result<Vec<ReceivedMessage>, SqError> {
|
||||
if !self.stream_started {
|
||||
self.start_subscribe().await?;
|
||||
self.stream_started = true;
|
||||
}
|
||||
|
||||
let frame = self.subscribe_conn.recv_frame().await?;
|
||||
|
||||
if frame.opcode == OP_SUBSCRIBE_END {
|
||||
return Ok(vec![]);
|
||||
}
|
||||
|
||||
if frame.opcode == OP_ERROR {
|
||||
let reader = codec::read_capnp(&frame.payload)
|
||||
.map_err(|e| SqError::Server(format!("decode error: {e}")))?;
|
||||
let err = reader
|
||||
.get_root::<data_plane_capnp::error_response::Reader>()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?;
|
||||
return Err(SqError::Server(
|
||||
err.get_message()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?
|
||||
.to_string()
|
||||
.map_err(|e| SqError::Server(format!("utf8 error: {e}")))?,
|
||||
));
|
||||
}
|
||||
|
||||
if frame.opcode != OP_SUBSCRIBE_RES {
|
||||
return Err(SqError::Server(format!(
|
||||
"unexpected opcode: 0x{:02x}",
|
||||
frame.opcode
|
||||
)));
|
||||
}
|
||||
|
||||
let reader = codec::read_capnp(&frame.payload)
|
||||
.map_err(|e| SqError::Server(format!("decode error: {e}")))?;
|
||||
let resp = reader
|
||||
.get_root::<data_plane_capnp::subscribe_response::Reader>()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?;
|
||||
|
||||
let messages = resp
|
||||
.get_messages()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?;
|
||||
|
||||
let mut result = Vec::with_capacity(messages.len() as usize);
|
||||
for i in 0..messages.len() {
|
||||
let m = messages.get(i);
|
||||
let headers_reader = m
|
||||
.get_headers()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?;
|
||||
|
||||
let mut headers = Vec::with_capacity(headers_reader.len() as usize);
|
||||
for j in 0..headers_reader.len() {
|
||||
let h = headers_reader.get(j);
|
||||
headers.push((
|
||||
h.get_key()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?
|
||||
.to_string()
|
||||
.map_err(|e| SqError::Server(format!("utf8 error: {e}")))?,
|
||||
h.get_value()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?
|
||||
.to_vec(),
|
||||
));
|
||||
}
|
||||
|
||||
result.push(ReceivedMessage {
|
||||
offset: m.get_offset(),
|
||||
topic: m
|
||||
.get_topic()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?
|
||||
.to_string()
|
||||
.map_err(|e| SqError::Server(format!("utf8 error: {e}")))?,
|
||||
partition: m.get_partition(),
|
||||
key: m
|
||||
.get_key()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?
|
||||
.to_vec(),
|
||||
value: m
|
||||
.get_value()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?
|
||||
.to_vec(),
|
||||
headers,
|
||||
timestamp_ms: m.get_timestamp_ms(),
|
||||
});
|
||||
}
|
||||
|
||||
if let Some(last) = result.last() {
|
||||
self.last_offset = Some(last.offset);
|
||||
|
||||
if self.config.auto_commit {
|
||||
let _ = self.commit_internal(last.offset).await;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(result)
|
||||
}
|
||||
|
||||
/// Manually commit an offset.
|
||||
pub async fn commit(&mut self, offset: u64) -> Result<(), SqError> {
|
||||
self.commit_internal(offset).await
|
||||
}
|
||||
|
||||
async fn start_subscribe(&mut self) -> Result<(), SqError> {
|
||||
let mut builder = capnp::message::Builder::new_default();
|
||||
{
|
||||
let mut req = builder.init_root::<data_plane_capnp::subscribe_request::Builder>();
|
||||
req.set_topic(&self.config.topic[..]);
|
||||
req.set_partition(self.config.partition);
|
||||
req.set_consumer_group(&self.config.consumer_group[..]);
|
||||
req.set_max_batch_size(self.config.max_poll_records);
|
||||
|
||||
if let Some(offset) = self.config.start_offset {
|
||||
req.set_start_offset(offset);
|
||||
req.set_has_start_offset(true);
|
||||
}
|
||||
}
|
||||
|
||||
let frame = codec::build_frame(OP_SUBSCRIBE_REQ, &builder);
|
||||
self.subscribe_conn.send_frame(frame).await
|
||||
}
|
||||
|
||||
async fn commit_internal(&mut self, offset: u64) -> Result<(), SqError> {
|
||||
let mut builder = capnp::message::Builder::new_default();
|
||||
{
|
||||
let mut req = builder.init_root::<data_plane_capnp::ack_request::Builder>();
|
||||
req.set_consumer_group(&self.config.consumer_group[..]);
|
||||
req.set_topic(&self.config.topic[..]);
|
||||
req.set_partition(self.config.partition);
|
||||
req.set_offset(offset);
|
||||
}
|
||||
|
||||
let frame = codec::build_frame(OP_ACK_REQ, &builder);
|
||||
self.ack_conn.send_frame(frame).await?;
|
||||
|
||||
let resp = self.ack_conn.recv_frame().await?;
|
||||
if resp.opcode == OP_ERROR {
|
||||
let reader = codec::read_capnp(&resp.payload)
|
||||
.map_err(|e| SqError::Server(format!("decode error: {e}")))?;
|
||||
let err = reader
|
||||
.get_root::<data_plane_capnp::error_response::Reader>()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?;
|
||||
return Err(SqError::Server(
|
||||
err.get_message()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?
|
||||
.to_string()
|
||||
.map_err(|e| SqError::Server(format!("utf8 error: {e}")))?,
|
||||
));
|
||||
}
|
||||
|
||||
if resp.opcode != OP_ACK_RES {
|
||||
return Err(SqError::Server(format!(
|
||||
"unexpected opcode: 0x{:02x}",
|
||||
resp.opcode
|
||||
)));
|
||||
}
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
145
crates/sq-sdk/src/capnp_producer.rs
Normal file
145
crates/sq-sdk/src/capnp_producer.rs
Normal file
@@ -0,0 +1,145 @@
|
||||
use sq_capnp_interface::codec::{self, OP_ERROR, OP_PUBLISH_REQ, OP_PUBLISH_RES};
|
||||
use sq_capnp_interface::data_plane_capnp;
|
||||
|
||||
use crate::capnp_connection::Connection;
|
||||
use crate::error::SqError;
|
||||
use crate::producer::{ProducerMessage, SendResult};
|
||||
use crate::types::AckMode;
|
||||
|
||||
/// Configuration for an SQ producer (Cap'n Proto transport).
|
||||
pub struct ProducerConfig {
|
||||
/// Server address (e.g., "127.0.0.1:6064").
|
||||
pub address: String,
|
||||
/// Acknowledgment mode.
|
||||
pub ack_mode: AckMode,
|
||||
/// Producer identifier.
|
||||
pub producer_id: String,
|
||||
}
|
||||
|
||||
impl Default for ProducerConfig {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
address: "127.0.0.1:6064".to_string(),
|
||||
ack_mode: AckMode::All,
|
||||
producer_id: "default".to_string(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// SQ producer using Cap'n Proto over TCP.
|
||||
pub struct Producer {
|
||||
conn: Connection,
|
||||
config: ProducerConfig,
|
||||
}
|
||||
|
||||
impl Producer {
|
||||
/// Connect to an SQ server and create a new producer.
|
||||
pub async fn connect(config: ProducerConfig) -> Result<Self, SqError> {
|
||||
let conn = Connection::connect(&config.address).await?;
|
||||
Ok(Self { conn, config })
|
||||
}
|
||||
|
||||
/// Send a single message.
|
||||
pub async fn send(
|
||||
&mut self,
|
||||
topic: &str,
|
||||
key: Option<&[u8]>,
|
||||
value: &[u8],
|
||||
) -> Result<SendResult, SqError> {
|
||||
let results = self
|
||||
.send_batch(vec![ProducerMessage {
|
||||
topic: topic.to_string(),
|
||||
key: key.map(|k| k.to_vec()),
|
||||
value: value.to_vec(),
|
||||
headers: Vec::new(),
|
||||
}])
|
||||
.await?;
|
||||
Ok(results.into_iter().next().unwrap())
|
||||
}
|
||||
|
||||
/// Send a batch of messages.
|
||||
pub async fn send_batch(
|
||||
&mut self,
|
||||
messages: Vec<ProducerMessage>,
|
||||
) -> Result<Vec<SendResult>, SqError> {
|
||||
// Build capnp request.
|
||||
let mut builder = capnp::message::Builder::new_default();
|
||||
{
|
||||
let mut req = builder.init_root::<data_plane_capnp::publish_request::Builder>();
|
||||
req.set_ack_mode(self.config.ack_mode.to_capnp_u8());
|
||||
req.set_producer_id(&self.config.producer_id[..]);
|
||||
|
||||
let mut msg_list = req.init_messages(messages.len() as u32);
|
||||
for (i, m) in messages.iter().enumerate() {
|
||||
let mut entry = msg_list.reborrow().get(i as u32);
|
||||
entry.set_topic(&m.topic[..]);
|
||||
entry.set_key(m.key.as_deref().unwrap_or(&[]));
|
||||
entry.set_value(&m.value);
|
||||
|
||||
let mut headers = entry.init_headers(m.headers.len() as u32);
|
||||
for (j, (k, v)) in m.headers.iter().enumerate() {
|
||||
let mut h = headers.reborrow().get(j as u32);
|
||||
h.set_key(&k[..]);
|
||||
h.set_value(v);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
let frame = codec::build_frame(OP_PUBLISH_REQ, &builder);
|
||||
self.conn.send_frame(frame).await?;
|
||||
|
||||
// Read response.
|
||||
let resp_frame = self.conn.recv_frame().await?;
|
||||
|
||||
if resp_frame.opcode == OP_ERROR {
|
||||
let msg = decode_error(&resp_frame.payload)?;
|
||||
return Err(SqError::Server(msg));
|
||||
}
|
||||
|
||||
if resp_frame.opcode != OP_PUBLISH_RES {
|
||||
return Err(SqError::Server(format!(
|
||||
"unexpected opcode: 0x{:02x}",
|
||||
resp_frame.opcode
|
||||
)));
|
||||
}
|
||||
|
||||
// Decode response.
|
||||
let reader = codec::read_capnp(&resp_frame.payload)
|
||||
.map_err(|e| SqError::Server(format!("decode error: {e}")))?;
|
||||
let resp = reader
|
||||
.get_root::<data_plane_capnp::publish_response::Reader>()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?;
|
||||
|
||||
let results = resp
|
||||
.get_results()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?;
|
||||
|
||||
let mut send_results = Vec::with_capacity(results.len() as usize);
|
||||
for i in 0..results.len() {
|
||||
let r = results.get(i);
|
||||
send_results.push(SendResult {
|
||||
topic: r
|
||||
.get_topic()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?
|
||||
.to_string()
|
||||
.map_err(|e| SqError::Server(format!("utf8 error: {e}")))?,
|
||||
partition: r.get_partition(),
|
||||
offset: r.get_offset(),
|
||||
});
|
||||
}
|
||||
|
||||
Ok(send_results)
|
||||
}
|
||||
}
|
||||
|
||||
fn decode_error(payload: &[u8]) -> Result<String, SqError> {
|
||||
let reader = codec::read_capnp(payload)
|
||||
.map_err(|e| SqError::Server(format!("decode error: {e}")))?;
|
||||
let err = reader
|
||||
.get_root::<data_plane_capnp::error_response::Reader>()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?;
|
||||
err.get_message()
|
||||
.map_err(|e| SqError::Server(format!("schema error: {e}")))?
|
||||
.to_string()
|
||||
.map_err(|e| SqError::Server(format!("utf8 error: {e}")))
|
||||
}
|
||||
24
crates/sq-sdk/src/connection.rs
Normal file
24
crates/sq-sdk/src/connection.rs
Normal file
@@ -0,0 +1,24 @@
|
||||
use crate::error::SqError;
|
||||
|
||||
/// Manages a gRPC channel to an SQ server.
|
||||
#[derive(Clone)]
|
||||
pub struct GrpcConnection {
|
||||
channel: tonic::transport::Channel,
|
||||
}
|
||||
|
||||
impl GrpcConnection {
|
||||
/// Connect to an SQ server at the given address (e.g., "http://127.0.0.1:6060").
|
||||
pub async fn connect(address: &str) -> Result<Self, SqError> {
|
||||
let channel = tonic::transport::Channel::from_shared(address.to_string())
|
||||
.map_err(|e| SqError::Connection(e.to_string()))?
|
||||
.connect()
|
||||
.await?;
|
||||
|
||||
Ok(Self { channel })
|
||||
}
|
||||
|
||||
/// Get the underlying tonic channel.
|
||||
pub fn channel(&self) -> tonic::transport::Channel {
|
||||
self.channel.clone()
|
||||
}
|
||||
}
|
||||
154
crates/sq-sdk/src/consumer.rs
Normal file
154
crates/sq-sdk/src/consumer.rs
Normal file
@@ -0,0 +1,154 @@
|
||||
use sq_grpc_interface::{
|
||||
data_plane_service_client::DataPlaneServiceClient, AckRequest, ConsumedMessage,
|
||||
SubscribeRequest, SubscribeResponse,
|
||||
};
|
||||
use tokio_stream::StreamExt;
|
||||
|
||||
use crate::connection::GrpcConnection;
|
||||
use crate::error::SqError;
|
||||
|
||||
/// Configuration for an SQ gRPC consumer.
|
||||
pub struct GrpcConsumerConfig {
|
||||
/// Server address (e.g., "http://127.0.0.1:6060").
|
||||
pub address: String,
|
||||
/// Consumer group name.
|
||||
pub consumer_group: String,
|
||||
/// Topic to consume from.
|
||||
pub topic: String,
|
||||
/// Partition to consume from.
|
||||
pub partition: u32,
|
||||
/// Whether to automatically commit offsets.
|
||||
pub auto_commit: bool,
|
||||
/// Maximum number of messages to receive per batch.
|
||||
pub max_poll_records: u32,
|
||||
}
|
||||
|
||||
impl Default for GrpcConsumerConfig {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
address: "http://127.0.0.1:6060".to_string(),
|
||||
consumer_group: "default".to_string(),
|
||||
topic: String::new(),
|
||||
partition: 0,
|
||||
auto_commit: true,
|
||||
max_poll_records: 100,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A message consumed from SQ.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct ReceivedMessage {
|
||||
pub offset: u64,
|
||||
pub topic: String,
|
||||
pub partition: u32,
|
||||
pub key: Vec<u8>,
|
||||
pub value: Vec<u8>,
|
||||
pub headers: Vec<(String, Vec<u8>)>,
|
||||
pub timestamp_ms: u64,
|
||||
}
|
||||
|
||||
impl From<ConsumedMessage> for ReceivedMessage {
|
||||
fn from(m: ConsumedMessage) -> Self {
|
||||
Self {
|
||||
offset: m.offset,
|
||||
topic: m.topic,
|
||||
partition: m.partition,
|
||||
key: m.key,
|
||||
value: m.value,
|
||||
headers: m.headers.into_iter().map(|h| (h.key, h.value)).collect(),
|
||||
timestamp_ms: m.timestamp_ms,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// SQ gRPC consumer client. Receives messages from an SQ server via streaming.
|
||||
pub struct GrpcConsumer {
|
||||
client: DataPlaneServiceClient<tonic::transport::Channel>,
|
||||
config: GrpcConsumerConfig,
|
||||
stream: Option<tonic::Streaming<SubscribeResponse>>,
|
||||
last_offset: Option<u64>,
|
||||
}
|
||||
|
||||
impl GrpcConsumer {
|
||||
/// Connect to an SQ server and create a new consumer.
|
||||
pub async fn connect(config: GrpcConsumerConfig) -> Result<Self, SqError> {
|
||||
let conn = GrpcConnection::connect(&config.address).await?;
|
||||
let client = DataPlaneServiceClient::new(conn.channel());
|
||||
|
||||
Ok(Self {
|
||||
client,
|
||||
config,
|
||||
stream: None,
|
||||
last_offset: None,
|
||||
})
|
||||
}
|
||||
|
||||
/// Poll for new messages. Establishes the subscription stream on first call.
|
||||
/// Returns an empty vec if no messages are available yet.
|
||||
pub async fn poll(&mut self) -> Result<Vec<ReceivedMessage>, SqError> {
|
||||
// Establish stream if not yet connected.
|
||||
if self.stream.is_none() {
|
||||
let response = self
|
||||
.client
|
||||
.subscribe(tonic::Request::new(SubscribeRequest {
|
||||
topic: self.config.topic.clone(),
|
||||
partition: self.config.partition,
|
||||
consumer_group: self.config.consumer_group.clone(),
|
||||
start_offset: None, // Uses committed offset if consumer group set.
|
||||
max_batch_size: self.config.max_poll_records,
|
||||
}))
|
||||
.await?;
|
||||
|
||||
self.stream = Some(response.into_inner());
|
||||
}
|
||||
|
||||
let stream = self.stream.as_mut().unwrap();
|
||||
|
||||
match stream.next().await {
|
||||
Some(Ok(response)) => {
|
||||
let messages: Vec<ReceivedMessage> = response
|
||||
.messages
|
||||
.into_iter()
|
||||
.map(ReceivedMessage::from)
|
||||
.collect();
|
||||
|
||||
if let Some(last) = messages.last() {
|
||||
self.last_offset = Some(last.offset);
|
||||
|
||||
// Auto-commit if enabled.
|
||||
if self.config.auto_commit {
|
||||
// Best-effort commit; don't fail the poll on commit error.
|
||||
let _ = self.commit_internal(last.offset).await;
|
||||
}
|
||||
}
|
||||
|
||||
Ok(messages)
|
||||
}
|
||||
Some(Err(status)) => Err(SqError::from(status)),
|
||||
None => {
|
||||
// Stream ended - reset so next poll reconnects.
|
||||
self.stream = None;
|
||||
Ok(vec![])
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Manually commit an offset for this consumer's group/topic/partition.
|
||||
pub async fn commit(&mut self, offset: u64) -> Result<(), SqError> {
|
||||
self.commit_internal(offset).await
|
||||
}
|
||||
|
||||
async fn commit_internal(&mut self, offset: u64) -> Result<(), SqError> {
|
||||
self.client
|
||||
.ack(tonic::Request::new(AckRequest {
|
||||
consumer_group: self.config.consumer_group.clone(),
|
||||
topic: self.config.topic.clone(),
|
||||
partition: self.config.partition,
|
||||
offset,
|
||||
}))
|
||||
.await?;
|
||||
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
34
crates/sq-sdk/src/error.rs
Normal file
34
crates/sq-sdk/src/error.rs
Normal file
@@ -0,0 +1,34 @@
|
||||
#[derive(Debug, thiserror::Error)]
|
||||
pub enum SqError {
|
||||
#[error("connection failed: {0}")]
|
||||
Connection(String),
|
||||
|
||||
#[error("server error: {0}")]
|
||||
Server(String),
|
||||
|
||||
#[error("invalid argument: {0}")]
|
||||
InvalidArgument(String),
|
||||
|
||||
#[error("not found: {0}")]
|
||||
NotFound(String),
|
||||
|
||||
#[error("stream ended")]
|
||||
StreamEnded,
|
||||
}
|
||||
|
||||
impl From<tonic::Status> for SqError {
|
||||
fn from(status: tonic::Status) -> Self {
|
||||
match status.code() {
|
||||
tonic::Code::InvalidArgument => SqError::InvalidArgument(status.message().to_string()),
|
||||
tonic::Code::NotFound => SqError::NotFound(status.message().to_string()),
|
||||
tonic::Code::Unavailable => SqError::Connection(status.message().to_string()),
|
||||
_ => SqError::Server(format!("{}: {}", status.code(), status.message())),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl From<tonic::transport::Error> for SqError {
|
||||
fn from(err: tonic::transport::Error) -> Self {
|
||||
SqError::Connection(err.to_string())
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,28 @@
|
||||
pub mod batch_producer;
|
||||
pub mod capnp_batch_producer;
|
||||
pub mod capnp_connection;
|
||||
pub mod capnp_consumer;
|
||||
pub mod capnp_producer;
|
||||
pub mod connection;
|
||||
pub mod consumer;
|
||||
pub mod error;
|
||||
pub mod producer;
|
||||
pub mod types;
|
||||
|
||||
// Default (capnp) types — these are the primary SDK interface.
|
||||
pub use capnp_batch_producer::{BatchProducer, BatchProducerConfig};
|
||||
pub use capnp_connection::Connection;
|
||||
pub use capnp_consumer::{Consumer, ConsumerConfig};
|
||||
pub use capnp_producer::{Producer, ProducerConfig};
|
||||
pub use types::AckMode;
|
||||
|
||||
// gRPC types (available but not the default transport).
|
||||
pub use batch_producer::{GrpcBatchProducer, GrpcBatchProducerConfig};
|
||||
pub use connection::GrpcConnection;
|
||||
pub use consumer::{GrpcConsumer, GrpcConsumerConfig};
|
||||
pub use producer::{GrpcProducer, GrpcProducerConfig};
|
||||
|
||||
// Shared types used by both transports.
|
||||
pub use consumer::ReceivedMessage;
|
||||
pub use error::SqError;
|
||||
pub use producer::{ProducerMessage, SendResult};
|
||||
|
||||
143
crates/sq-sdk/src/producer.rs
Normal file
143
crates/sq-sdk/src/producer.rs
Normal file
@@ -0,0 +1,143 @@
|
||||
use sq_grpc_interface::{
|
||||
data_plane_service_client::DataPlaneServiceClient, AckMode, MessageHeader, PublishMessage,
|
||||
PublishRequest, PublishSettings,
|
||||
};
|
||||
|
||||
use crate::connection::GrpcConnection;
|
||||
use crate::error::SqError;
|
||||
|
||||
/// Configuration for an SQ gRPC producer.
|
||||
pub struct GrpcProducerConfig {
|
||||
/// Server address (e.g., "http://127.0.0.1:6060").
|
||||
pub address: String,
|
||||
/// Default ack mode for publish requests.
|
||||
pub ack_mode: AckMode,
|
||||
/// Producer identifier.
|
||||
pub producer_id: String,
|
||||
}
|
||||
|
||||
impl Default for GrpcProducerConfig {
|
||||
fn default() -> Self {
|
||||
Self {
|
||||
address: "http://127.0.0.1:6060".to_string(),
|
||||
ack_mode: AckMode::All,
|
||||
producer_id: "default".to_string(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// Result of sending a single message.
|
||||
#[derive(Debug, Clone)]
|
||||
pub struct SendResult {
|
||||
pub topic: String,
|
||||
pub partition: u32,
|
||||
pub offset: u64,
|
||||
}
|
||||
|
||||
/// A message to be sent by the producer.
|
||||
pub struct ProducerMessage {
|
||||
pub topic: String,
|
||||
pub key: Option<Vec<u8>>,
|
||||
pub value: Vec<u8>,
|
||||
pub headers: Vec<(String, Vec<u8>)>,
|
||||
}
|
||||
|
||||
impl ProducerMessage {
|
||||
pub fn new(topic: impl Into<String>, value: impl Into<Vec<u8>>) -> Self {
|
||||
Self {
|
||||
topic: topic.into(),
|
||||
key: None,
|
||||
value: value.into(),
|
||||
headers: Vec::new(),
|
||||
}
|
||||
}
|
||||
|
||||
pub fn with_key(mut self, key: impl Into<Vec<u8>>) -> Self {
|
||||
self.key = Some(key.into());
|
||||
self
|
||||
}
|
||||
|
||||
pub fn with_header(mut self, key: impl Into<String>, value: impl Into<Vec<u8>>) -> Self {
|
||||
self.headers.push((key.into(), value.into()));
|
||||
self
|
||||
}
|
||||
}
|
||||
|
||||
/// SQ gRPC producer client. Sends messages to an SQ server.
|
||||
pub struct GrpcProducer {
|
||||
client: DataPlaneServiceClient<tonic::transport::Channel>,
|
||||
config: GrpcProducerConfig,
|
||||
}
|
||||
|
||||
impl GrpcProducer {
|
||||
/// Connect to an SQ server and create a new producer.
|
||||
pub async fn connect(config: GrpcProducerConfig) -> Result<Self, SqError> {
|
||||
let conn = GrpcConnection::connect(&config.address).await?;
|
||||
let client = DataPlaneServiceClient::new(conn.channel());
|
||||
|
||||
Ok(Self { client, config })
|
||||
}
|
||||
|
||||
/// Send a single message.
|
||||
pub async fn send(
|
||||
&mut self,
|
||||
topic: &str,
|
||||
key: Option<&[u8]>,
|
||||
value: &[u8],
|
||||
) -> Result<SendResult, SqError> {
|
||||
let results = self
|
||||
.send_batch(vec![ProducerMessage {
|
||||
topic: topic.to_string(),
|
||||
key: key.map(|k| k.to_vec()),
|
||||
value: value.to_vec(),
|
||||
headers: Vec::new(),
|
||||
}])
|
||||
.await?;
|
||||
|
||||
Ok(results.into_iter().next().unwrap())
|
||||
}
|
||||
|
||||
/// Send a batch of messages.
|
||||
pub async fn send_batch(
|
||||
&mut self,
|
||||
messages: Vec<ProducerMessage>,
|
||||
) -> Result<Vec<SendResult>, SqError> {
|
||||
let publish_messages: Vec<PublishMessage> = messages
|
||||
.into_iter()
|
||||
.map(|m| PublishMessage {
|
||||
topic: m.topic,
|
||||
key: m.key.unwrap_or_default(),
|
||||
value: m.value,
|
||||
headers: m
|
||||
.headers
|
||||
.into_iter()
|
||||
.map(|(k, v)| MessageHeader { key: k, value: v })
|
||||
.collect(),
|
||||
})
|
||||
.collect();
|
||||
|
||||
let response = self
|
||||
.client
|
||||
.publish(tonic::Request::new(PublishRequest {
|
||||
messages: publish_messages,
|
||||
settings: Some(PublishSettings {
|
||||
ack_mode: self.config.ack_mode.into(),
|
||||
}),
|
||||
producer_id: self.config.producer_id.clone(),
|
||||
}))
|
||||
.await?;
|
||||
|
||||
let results = response
|
||||
.into_inner()
|
||||
.results
|
||||
.into_iter()
|
||||
.map(|r| SendResult {
|
||||
topic: r.topic,
|
||||
partition: r.partition,
|
||||
offset: r.offset,
|
||||
})
|
||||
.collect();
|
||||
|
||||
Ok(results)
|
||||
}
|
||||
}
|
||||
37
crates/sq-sdk/src/types.rs
Normal file
37
crates/sq-sdk/src/types.rs
Normal file
@@ -0,0 +1,37 @@
|
||||
/// Acknowledgment mode for publish requests.
|
||||
#[derive(Debug, Clone, Copy, PartialEq, Eq)]
|
||||
pub enum AckMode {
|
||||
/// Wait for all replicas to acknowledge.
|
||||
All,
|
||||
/// Wait for the local replica only.
|
||||
Local,
|
||||
/// Fire-and-forget, no acknowledgment.
|
||||
None,
|
||||
}
|
||||
|
||||
impl AckMode {
|
||||
/// Convert to the wire `u8` used by the capnp protocol.
|
||||
pub fn to_capnp_u8(self) -> u8 {
|
||||
match self {
|
||||
AckMode::All => 1,
|
||||
AckMode::Local => 2,
|
||||
AckMode::None => 3,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl From<AckMode> for sq_grpc_interface::AckMode {
|
||||
fn from(mode: AckMode) -> Self {
|
||||
match mode {
|
||||
AckMode::All => sq_grpc_interface::AckMode::All,
|
||||
AckMode::Local => sq_grpc_interface::AckMode::Local,
|
||||
AckMode::None => sq_grpc_interface::AckMode::None,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Default for AckMode {
|
||||
fn default() -> Self {
|
||||
AckMode::All
|
||||
}
|
||||
}
|
||||
Reference in New Issue
Block a user