linguoxuan commented on code in PR #216: URL: https://github.com/apache/fluss-rust/pull/216#discussion_r2749200718
########## crates/fluss/src/client/lookup/lookup_sender.rs: ########## @@ -0,0 +1,444 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you 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. + +//! Lookup sender that processes batched lookup requests. +//! +//! The sender runs as a background task, draining lookups from the queue, +//! grouping them by destination server, and sending batched requests. + +use super::{LookupQuery, LookupQueue}; +use crate::client::metadata::Metadata; +use crate::error::{Error, FlussError, Result}; +use crate::metadata::TableBucket; +use crate::proto::LookupResponse; +use crate::rpc::message::LookupRequest; +use log::{debug, error, warn}; +use std::collections::HashMap; +use std::sync::Arc; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::time::Duration; +use tokio::sync::{Semaphore, mpsc}; + +/// Lookup sender that batches and sends lookup requests. +pub struct LookupSender { + /// Metadata for leader lookup + metadata: Arc<Metadata>, + /// The lookup queue to drain from + queue: LookupQueue, + /// Channel to re-enqueue failed lookups + re_enqueue_tx: mpsc::UnboundedSender<LookupQuery>, + /// Semaphore to limit in-flight requests + inflight_semaphore: Arc<Semaphore>, + /// Maximum number of retries + max_retries: i32, + /// Whether the sender is running + running: AtomicBool, + /// Whether to force close (abandon pending lookups) + force_close: AtomicBool, +} + +/// A batch of lookups going to the same table bucket. +struct LookupBatch { + table_bucket: TableBucket, + lookups: Vec<LookupQuery>, + keys: Vec<Vec<u8>>, +} + +impl LookupBatch { + fn new(table_bucket: TableBucket) -> Self { + Self { + table_bucket, + lookups: Vec::new(), + keys: Vec::new(), + } + } + + fn add_lookup(&mut self, lookup: LookupQuery) { + self.keys.push(lookup.key().to_vec()); + self.lookups.push(lookup); + } + + fn complete(&mut self, values: Vec<Option<Vec<u8>>>) { + if values.len() != self.lookups.len() { + let err_msg = format!( + "The number of return values ({}) does not match the number of lookups ({})", + values.len(), + self.lookups.len() + ); + for lookup in &mut self.lookups { + lookup.complete(Err(Error::UnexpectedError { + message: err_msg.clone(), + source: None, + })); + } + return; + } + + for (lookup, value) in self.lookups.iter_mut().zip(values.into_iter()) { + lookup.complete(Ok(value)); + } + } + + fn complete_exceptionally(&mut self, error_msg: &str) { + for lookup in &mut self.lookups { + lookup.complete(Err(Error::UnexpectedError { + message: error_msg.to_string(), + source: None, + })); + } + } +} + +impl LookupSender { + /// Creates a new lookup sender. + pub fn new( + metadata: Arc<Metadata>, + queue: LookupQueue, + re_enqueue_tx: mpsc::UnboundedSender<LookupQuery>, + max_inflight_requests: usize, + max_retries: i32, + ) -> Self { + Self { + metadata, + queue, + re_enqueue_tx, + inflight_semaphore: Arc::new(Semaphore::new(max_inflight_requests)), + max_retries, + running: AtomicBool::new(true), + force_close: AtomicBool::new(false), + } + } + + /// Runs the sender loop. + pub async fn run(&mut self) { + debug!("Starting Fluss lookup sender"); + + while self.running.load(Ordering::Acquire) { + if let Err(e) = self.run_once(false).await { + error!("Error in lookup sender: {}", e); + } + } + + debug!("Beginning shutdown of lookup sender, sending remaining lookups"); + + // Process remaining lookups during shutdown + if !self.force_close.load(Ordering::Acquire) && self.queue.has_undrained() { + if let Err(e) = self.run_once(true).await { + error!("Error during lookup sender shutdown: {}", e); + } + } + + debug!("Lookup sender shutdown complete"); + } + + /// Runs a single iteration of the sender loop. + async fn run_once(&mut self, drain_all: bool) -> Result<()> { + let lookups = if drain_all { + self.queue.drain_all() + } else { + self.queue.drain().await + }; + + self.send_lookups(lookups).await + } + + /// Groups and sends lookups to appropriate servers. + async fn send_lookups(&self, lookups: Vec<LookupQuery>) -> Result<()> { Review Comment: Currently, all errors are comsumed internally. Therefore, this method will not return any errors. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
