Copilot commented on code in PR #216: URL: https://github.com/apache/fluss-rust/pull/216#discussion_r2752706946
########## crates/fluss/src/client/lookup/lookup_client.rs: ########## @@ -0,0 +1,204 @@ +// 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 client that batches multiple lookups together for improved throughput. +//! +//! This client achieves parity with the Java client by: +//! - Queuing lookup operations instead of sending them immediately +//! - Batching multiple lookups to the same server/bucket +//! - Running a background sender task to process batches + +use super::{LookupQuery, LookupQueue}; +use crate::client::lookup::lookup_sender::LookupSender; +use crate::client::metadata::Metadata; +use crate::config::Config; +use crate::error::{Error, Result}; +use crate::metadata::{TableBucket, TablePath}; +use bytes::Bytes; +use log::{debug, error}; +use std::sync::Arc; +use std::sync::atomic::{AtomicBool, Ordering}; +use std::time::Duration; +use tokio::sync::mpsc; +use tokio::task::JoinHandle; + +/// A client that lookups values from the server with batching support. +/// +/// The lookup client uses a queue and background sender to batch multiple +/// lookup operations together, reducing network round trips and improving +/// throughput. +/// +/// # Example +/// +/// ```ignore +/// let lookup_client = LookupClient::new(config, metadata); +/// let result = lookup_client.lookup(table_path, table_bucket, key_bytes).await?; +/// ``` +pub struct LookupClient { + /// Channel to send lookup requests to the queue + lookup_tx: mpsc::Sender<LookupQuery>, + /// Handle to the sender task + sender_handle: Option<JoinHandle<()>>, + /// Shutdown signal sender + shutdown_tx: mpsc::Sender<()>, + /// Whether the client is closed + closed: AtomicBool, +} + +impl LookupClient { + /// Creates a new lookup client. + pub fn new(config: &Config, metadata: Arc<Metadata>) -> Self { + // Extract configuration values + let queue_size = config.lookup_queue_size; + let max_batch_size = config.lookup_max_batch_size; + let batch_timeout_ms = config.lookup_batch_timeout_ms; + let max_inflight = config.lookup_max_inflight_requests; + let max_retries = config.lookup_max_retries; + + // Create queue and channels + let (queue, lookup_tx, re_enqueue_tx) = + LookupQueue::new(queue_size, max_batch_size, batch_timeout_ms); + + // Create sender + let mut sender = + LookupSender::new(metadata, queue, re_enqueue_tx, max_inflight, max_retries); + + // Create shutdown channel + let (shutdown_tx, mut shutdown_rx) = mpsc::channel(1); + + // Spawn sender task + let sender_handle = tokio::spawn(async move { + tokio::select! { + _ = sender.run() => { + debug!("Lookup sender completed"); + } + _ = shutdown_rx.recv() => { + debug!("Lookup sender received shutdown signal"); + sender.initiate_close(); + } Review Comment: There's a potential issue with the shutdown logic. When `shutdown_rx.recv()` completes (line 89), `tokio::select!` will drop/cancel the `sender.run()` future. This means line 91 calls `sender.initiate_close()` on a sender that has already been cancelled and dropped. The `initiate_close()` call has no effect since the sender loop is no longer running. This differs from the writer_client.rs pattern (line 67) which calls `.await` on the close method. Consider restructuring to ensure graceful shutdown: the sender should handle the shutdown signal internally, or the select should not cancel sender.run(). ########## crates/fluss/src/client/lookup/lookup_sender.rs: ########## @@ -0,0 +1,453 @@ +// 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 crate::{BucketId, PartitionId, TableId}; +use bytes::Bytes; +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<Bytes>, +} + +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().clone()); + 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 + // TODO: Check the in flight request count in the accumulator. + 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); + } + } + + // TODO: If force close failed, add logic to abort incomplete lookup requests. + 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<()> { + if lookups.is_empty() { + return Ok(()); + } + + // Group by leader + let lookup_batches = self.group_by_leader(lookups); + + // If no lookup batches, sleep a bit to avoid busy loop. This case will happen when there is + // no leader for all the lookup request in queue. + if lookup_batches.is_empty() && !self.queue.has_undrained() { + // TODO: May use wait/notify mechanism to avoid active sleep, and use a dynamic sleep time based on the request waited time. + tokio::time::sleep(Duration::from_millis(100)).await; + return Ok(()); + } + + // Send batches to each destination + for (destination, batches) in lookup_batches { + self.send_lookup_request(destination, batches).await; + } + + Ok(()) + } + + /// Groups lookups by leader server. + fn group_by_leader( + &self, + lookups: Vec<LookupQuery>, + ) -> HashMap<i32, HashMap<TableBucket, LookupBatch>> { + let cluster = self.metadata.get_cluster(); + let mut batches_by_leader: HashMap<i32, HashMap<TableBucket, LookupBatch>> = HashMap::new(); + + for lookup in lookups { + let table_bucket = lookup.table_bucket().clone(); + + // TODO: Metadata requests are being sent too frequently here. consider first + // collecting the tables that need to be updated and then sending them together in + // one request. + let leader = match cluster.leader_for(&table_bucket) { + Some(leader) => leader.id(), + None => { + warn!( + "No leader found for table bucket {} during lookup", + table_bucket + ); + self.re_enqueue_lookup(lookup); + continue; + } + }; + + batches_by_leader + .entry(leader) + .or_default() + .entry(table_bucket.clone()) + .or_insert_with(|| LookupBatch::new(table_bucket)) + .add_lookup(lookup); + } + + batches_by_leader + } + + /// Sends lookup requests to a specific destination server. + async fn send_lookup_request( + &self, + destination: i32, + batches_by_bucket: HashMap<TableBucket, LookupBatch>, + ) { + // Group by table_id for request batching + let mut batches_by_table: HashMap<TableId, Vec<LookupBatch>> = HashMap::new(); + for (table_bucket, batch) in batches_by_bucket { + batches_by_table + .entry(table_bucket.table_id()) + .or_default() + .push(batch); + } + + let cluster = self.metadata.get_cluster(); + let tablet_server = match cluster.get_tablet_server(destination) { + Some(server) => server.clone(), + None => { + let err_msg = format!("Server {} is not found in metadata cache", destination); + for batches in batches_by_table.into_values() { + for mut batch in batches { + self.handle_lookup_error(&err_msg, true, &mut batch); + } + } + return; + } + }; + + let connection = match self.metadata.get_connection(&tablet_server).await { + Ok(conn) => conn, + Err(e) => { + let err_msg = format!("Failed to get connection to server {}: {}", destination, e); + for batches in batches_by_table.into_values() { + for mut batch in batches { + self.handle_lookup_error(&err_msg, true, &mut batch); + } + } + return; + } + }; + + // Send requests for each table + for (table_id, mut batches) in batches_by_table { + // Build the request with all buckets for this table + // Use std::mem::take to move keys instead of cloning to avoid deep copy overhead + let mut all_keys_by_bucket: Vec<(BucketId, Option<PartitionId>, Vec<Bytes>)> = + Vec::new(); + for batch in &mut batches { + all_keys_by_bucket.push(( + batch.table_bucket.bucket_id(), + batch.table_bucket.partition_id(), + std::mem::take(&mut batch.keys), + )); + } + + // Create lookup request for all buckets in this table + let request = LookupRequest::new_batched(table_id, all_keys_by_bucket); + + // Acquire semaphore permit + let _permit = match self.inflight_semaphore.clone().acquire_owned().await { + Ok(permit) => permit, + Err(_) => { + error!("Semaphore closed during lookup"); + for batch in &mut batches { + batch.complete_exceptionally("Lookup sender shutdown"); + } + return; + } + }; + + // Send request and handle response + match connection.request(request).await { + Ok(response) => { + self.handle_lookup_response(destination, response, &mut batches); + } + Err(e) => { + let err_msg = format!("Lookup request failed: {}", e); + let is_retriable = Self::is_retriable_error(&e); + for batch in &mut batches { + self.handle_lookup_error(&err_msg, is_retriable, batch); + } + } + } + } + } + + /// Checks if an error is retriable. + fn is_retriable_error(error: &Error) -> bool { + match error { + Error::LeaderNotAvailable { .. } => true, + Error::FlussAPIError { api_error } => { + let fluss_error = FlussError::for_code(api_error.code); + fluss_error.is_retriable() + } + _ => false, + } + } + + /// Handles the lookup response. + fn handle_lookup_response( + &self, + destination: i32, + response: LookupResponse, + batches: &mut [LookupBatch], + ) { + // Create a map from bucket_id to batch index for quick lookup + let bucket_to_index: HashMap<BucketId, usize> = batches + .iter() + .enumerate() + .map(|(idx, batch)| (batch.table_bucket.bucket_id(), idx)) + .collect(); + + // Track which batches have been processed + let mut processed_batches = vec![false; batches.len()]; + + for bucket_resp in response.buckets_resp { + let bucket_id = bucket_resp.bucket_id; + if let Some(&batch_idx) = bucket_to_index.get(&bucket_id) { + processed_batches[batch_idx] = true; + let batch = &mut batches[batch_idx]; + + // Check for errors + if let Some(error_code) = bucket_resp.error_code { + let fluss_error = FlussError::for_code(error_code); + if fluss_error != FlussError::None { + let err_msg = format!( + "Lookup error for bucket {}: code={}, message={}", + bucket_id, + error_code, + bucket_resp.error_message.unwrap_or_default() + ); + let is_retriable = fluss_error.is_retriable(); + self.handle_lookup_error(&err_msg, is_retriable, batch); + continue; + } + } + + // Extract values + let values: Vec<Option<Vec<u8>>> = bucket_resp + .values + .into_iter() + .map(|pb_value| pb_value.values) + .collect(); + + batch.complete(values); + } else { + error!( + "Received response for unknown bucket {} from server {}", + bucket_id, destination + ); + } + } + + // Handle any batches that were not included in the response + for (idx, processed) in processed_batches.iter().enumerate() { + if !processed { + let batch = &mut batches[idx]; + // If the batch has lookups that haven't been processed, retry them + if !batch.lookups.is_empty() { + let err_msg = format!( + "Bucket {} response missing from server {}", + batch.table_bucket.bucket_id(), + destination + ); + // Treat missing bucket response as retriable + self.handle_lookup_error(&err_msg, true, batch); + } + } + } + } + + /// Handles lookup errors with retry logic. + fn handle_lookup_error(&self, error_msg: &str, is_retriable: bool, batch: &mut LookupBatch) { + let mut lookups_to_retry = Vec::new(); + let mut lookups_to_complete = Vec::new(); + + for lookup in batch.lookups.drain(..) { + if is_retriable && lookup.retries() < self.max_retries && !lookup.is_done() { + warn!( + "Lookup error for bucket {}, retrying ({} attempts left): {}", + batch.table_bucket, + self.max_retries - lookup.retries(), + error_msg + ); + lookups_to_retry.push(lookup); + } else { + lookups_to_complete.push(lookup); + } + } + + // Re-enqueue retriable lookups + for lookup in lookups_to_retry { + lookup.increment_retries(); + self.re_enqueue_lookup(lookup); + } + + // Complete non-retriable lookups with error + for mut lookup in lookups_to_complete { + warn!( + "Lookup failed for bucket {}: {}", + batch.table_bucket, error_msg + ); + lookup.complete(Err(Error::UnexpectedError { + message: error_msg.to_string(), + source: None, + })); + } + } + + /// Re-enqueues a lookup for retry. + fn re_enqueue_lookup(&self, lookup: LookupQuery) { + if let Err(e) = self.re_enqueue_tx.send(lookup) { + error!("Failed to re-enqueue lookup: {}", e); Review Comment: When re-enqueuing fails (line 437), the error is only logged but the lookup is not completed. This means the caller waiting for the lookup result will hang indefinitely since the oneshot channel never receives a response. Consider completing the lookup with an error when re-enqueuing fails. ```suggestion error!("Failed to re-enqueue lookup: {}", e); // Ensure the caller does not hang by completing the lookup with an error. let mut failed_lookup = e.into_inner(); failed_lookup.complete(Err(Error::UnexpectedError { message: "Failed to re-enqueue lookup for retry".to_string(), source: None, })); ``` ########## crates/fluss/src/client/lookup/lookup_queue.rs: ########## @@ -0,0 +1,138 @@ +// 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 queue for buffering pending lookup operations. +//! +//! This queue buffers lookup operations and provides batched draining +//! to improve throughput by reducing network round trips. + +use super::LookupQuery; +use std::time::Duration; +use tokio::sync::mpsc; +use tokio::time::timeout; + +/// A queue that buffers pending lookup operations and provides batched draining. +/// +/// The queue supports two types of entries: +/// - New lookups from client calls +/// - Re-enqueued lookups from retry logic +/// +/// Re-enqueued lookups are prioritized over new lookups to ensure fair processing. +pub struct LookupQueue { + /// Channel for receiving lookup requests + lookup_rx: mpsc::Receiver<LookupQuery>, + /// Channel for receiving re-enqueued lookups + re_enqueue_rx: mpsc::UnboundedReceiver<LookupQuery>, + /// Maximum batch size for draining + max_batch_size: usize, + /// Timeout for batch collection + batch_timeout: Duration, +} + +impl LookupQueue { + /// Creates a new lookup queue with the specified configuration. + pub fn new( + queue_size: usize, + max_batch_size: usize, + batch_timeout_ms: u64, + ) -> ( + Self, + mpsc::Sender<LookupQuery>, + mpsc::UnboundedSender<LookupQuery>, + ) { + let (lookup_tx, lookup_rx) = mpsc::channel(queue_size); + let (re_enqueue_tx, re_enqueue_rx) = mpsc::unbounded_channel(); + + let queue = Self { + lookup_rx, + re_enqueue_rx, + max_batch_size, + batch_timeout: Duration::from_millis(batch_timeout_ms), + }; + + (queue, lookup_tx, re_enqueue_tx) + } + + /// Drains a batch of lookup queries from the queue. + pub async fn drain(&mut self) -> Vec<LookupQuery> { + let mut lookups = Vec::with_capacity(self.max_batch_size); + let deadline = tokio::time::Instant::now() + self.batch_timeout; + + loop { + let remaining = deadline.saturating_duration_since(tokio::time::Instant::now()); + if remaining.is_zero() { + break; + } + + // First drain re-enqueued lookups (prioritized) + while lookups.len() < self.max_batch_size { + match self.re_enqueue_rx.try_recv() { + Ok(lookup) => lookups.push(lookup), + Err(_) => break, + } + } + + if lookups.len() >= self.max_batch_size { + break; + } Review Comment: The drain logic prioritizes re-enqueued lookups over new lookups from the main queue. If re-enqueued lookups fill the batch (line 82-87), the method breaks immediately (line 90) without attempting to receive from the main lookup queue (line 94). This could lead to starvation of new lookup requests when there are many retries, as retry lookups will always be processed first and could continuously fill batches. Consider interleaving re-enqueued and new lookups or setting a limit on the proportion of re-enqueued items per batch to ensure fairness. ########## crates/fluss/src/client/lookup/lookup_sender.rs: ########## @@ -0,0 +1,453 @@ +// 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 crate::{BucketId, PartitionId, TableId}; +use bytes::Bytes; +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<Bytes>, +} + +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().clone()); + 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 + // TODO: Check the in flight request count in the accumulator. + 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); + } + } + + // TODO: If force close failed, add logic to abort incomplete lookup requests. + 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<()> { + if lookups.is_empty() { + return Ok(()); + } + + // Group by leader + let lookup_batches = self.group_by_leader(lookups); + + // If no lookup batches, sleep a bit to avoid busy loop. This case will happen when there is + // no leader for all the lookup request in queue. + if lookup_batches.is_empty() && !self.queue.has_undrained() { + // TODO: May use wait/notify mechanism to avoid active sleep, and use a dynamic sleep time based on the request waited time. + tokio::time::sleep(Duration::from_millis(100)).await; + return Ok(()); + } + + // Send batches to each destination + for (destination, batches) in lookup_batches { + self.send_lookup_request(destination, batches).await; + } + + Ok(()) + } + + /// Groups lookups by leader server. + fn group_by_leader( + &self, + lookups: Vec<LookupQuery>, + ) -> HashMap<i32, HashMap<TableBucket, LookupBatch>> { + let cluster = self.metadata.get_cluster(); + let mut batches_by_leader: HashMap<i32, HashMap<TableBucket, LookupBatch>> = HashMap::new(); + + for lookup in lookups { + let table_bucket = lookup.table_bucket().clone(); + + // TODO: Metadata requests are being sent too frequently here. consider first + // collecting the tables that need to be updated and then sending them together in + // one request. + let leader = match cluster.leader_for(&table_bucket) { Review Comment: This code uses `cluster.leader_for()` directly, which doesn't trigger metadata updates when a leader is not found. The old implementation in `lookup.rs` used `metadata.leader_for()` which includes retry logic and automatically updates metadata. Without metadata updates, lookups will fail repeatedly when leaders are not in the cache, even though they might be available after a metadata refresh. Consider updating the lookup_sender to trigger metadata updates when leaders are not found, similar to how it's done in other parts of the codebase. -- 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]
