leekeiabstraction commented on code in PR #216:
URL: https://github.com/apache/fluss-rust/pull/216#discussion_r2745624585


##########
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);
+            }
+        }
+

Review Comment:
   Should we also port over the TODO comments from Java side?



##########
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:
   Is an Err ever returned from within this method?



##########
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<()> {
+        if lookups.is_empty() {
+            return Ok(());
+        }
+
+        // Group by leader
+        let lookup_batches = self.group_by_leader(lookups);
+
+        if lookup_batches.is_empty() && !self.queue.has_undrained() {
+            // No lookups to send and queue is empty, sleep to avoid busy loop
+            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();
+
+            // Find leader for this bucket
+            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<i64, Vec<LookupBatch>> = 
HashMap::new();

Review Comment:
   Nit: Use TableId type.



##########
crates/fluss/src/client/lookup/lookup_queue.rs:
##########
@@ -0,0 +1,130 @@
+// 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> {

Review Comment:
   This implementation seems to subtly differ from java:
   
   Java side: Loop repeatedly through re-enqueud and queued until batch size is 
reached or deadline reached.
   
   Rust side: Loop once through re-enqueued and then repeatedly through queued 
until batch size or deadline reached.
   
   The java side always prioritise re-enqueued. Is my understanding correct and 
if so what's the motivation of the subtle difference? I think Java side is 
safer as it will always drain the re-enqueued which is unbounded and let queue 
back pressure take effect.



##########
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<()> {
+        if lookups.is_empty() {
+            return Ok(());
+        }
+
+        // Group by leader
+        let lookup_batches = self.group_by_leader(lookups);
+
+        if lookup_batches.is_empty() && !self.queue.has_undrained() {
+            // No lookups to send and queue is empty, sleep to avoid busy loop
+            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();
+
+            // Find leader for this bucket
+            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<i64, 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<(i32, Option<i64>, Vec<Vec<u8>>)> 
= Vec::new();

Review Comment:
   Let use respective types, make for easier reading.



##########
crates/fluss/src/client/lookup/lookup_client.rs:
##########
@@ -0,0 +1,194 @@
+// 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 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();
+                }
+            }
+        });
+
+        Self {
+            lookup_tx,
+            sender_handle: Some(sender_handle),
+            shutdown_tx,
+            closed: AtomicBool::new(false),
+        }
+    }
+
+    /// Looks up a value by its primary key.
+    ///
+    /// This method queues the lookup operation and returns a future that will
+    /// complete when the server responds. Multiple lookups may be batched 
together
+    /// for improved throughput.
+    ///
+    /// # Arguments
+    /// * `table_path` - The table path
+    /// * `table_bucket` - The table bucket
+    /// * `key_bytes` - The encoded primary key bytes
+    ///
+    /// # Returns
+    /// * `Ok(Some(bytes))` - The value bytes if found
+    /// * `Ok(None)` - If the key was not found
+    /// * `Err(Error)` - If the lookup fails
+    pub async fn lookup(
+        &self,
+        table_path: TablePath,
+        table_bucket: TableBucket,
+        key_bytes: Vec<u8>,
+    ) -> Result<Option<Vec<u8>>> {
+        // Check if the client is closed
+        if self.closed.load(Ordering::Acquire) {
+            return Err(Error::UnexpectedError {
+                message: "Lookup client is closed".to_string(),
+                source: None,
+            });
+        }
+
+        let (result_tx, result_rx) = tokio::sync::oneshot::channel();
+
+        let query = LookupQuery::new(table_path, table_bucket, key_bytes, 
result_tx);
+
+        // Send to queue
+        self.lookup_tx
+            .send(query)
+            .await
+            .map_err(|_| Error::UnexpectedError {
+                message: "Failed to queue lookup: channel closed".to_string(),

Review Comment:
   Should we log out the details of the failed query?



-- 
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]


Reply via email to