Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
15 changes: 15 additions & 0 deletions crates/fluss/src/client/connection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

use crate::client::WriterClient;
use crate::client::admin::FlussAdmin;
use crate::client::lookup::LookupClient;
use crate::client::metadata::Metadata;
use crate::client::table::FlussTable;
use crate::config::Config;
Expand All @@ -32,6 +33,7 @@ pub struct FlussConnection {
network_connects: Arc<RpcClient>,
args: Config,
writer_client: RwLock<Option<Arc<WriterClient>>>,
lookup_client: RwLock<Option<Arc<LookupClient>>>,
}

impl FlussConnection {
Expand All @@ -48,6 +50,7 @@ impl FlussConnection {
network_connects: connections.clone(),
args: arg.clone(),
writer_client: Default::default(),
lookup_client: Default::default(),
})
}

Expand Down Expand Up @@ -78,6 +81,18 @@ impl FlussConnection {
Ok(client)
}

/// Gets or creates a lookup client for batched lookup operations.
pub fn get_or_create_lookup_client(&self) -> Arc<LookupClient> {
if let Some(client) = self.lookup_client.read().as_ref() {
return client.clone();
}

// If not exists, create new one
let client = Arc::new(LookupClient::new(&self.args, self.metadata.clone()));
*self.lookup_client.write() = Some(client.clone());
client
}

pub async fn get_table(&self, table_path: &TablePath) -> Result<FlussTable<'_>> {
self.metadata.update_table_metadata(table_path).await?;
let table_info = self.metadata.get_cluster().get_table(table_path).clone();
Expand Down
188 changes: 188 additions & 0 deletions crates/fluss/src/client/lookup/lookup_client.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,188 @@
// 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::lookup_queue::{
DEFAULT_LOOKUP_BATCH_TIMEOUT_MS, DEFAULT_LOOKUP_MAX_BATCH_SIZE, DEFAULT_LOOKUP_QUEUE_SIZE,
};
use super::lookup_sender::{DEFAULT_LOOKUP_MAX_INFLIGHT_REQUESTS, DEFAULT_LOOKUP_MAX_RETRIES};
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::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: JoinHandle<()>,
/// Shutdown signal sender
shutdown_tx: mpsc::Sender<()>,
}

impl LookupClient {
/// Creates a new lookup client.
pub fn new(config: &Config, metadata: Arc<Metadata>) -> Self {
// Extract configuration values (use defaults if not configured)
let queue_size = config
.lookup_queue_size
.unwrap_or(DEFAULT_LOOKUP_QUEUE_SIZE);
let max_batch_size = config
.lookup_max_batch_size
.unwrap_or(DEFAULT_LOOKUP_MAX_BATCH_SIZE);
let batch_timeout_ms = config
.lookup_batch_timeout_ms
.unwrap_or(DEFAULT_LOOKUP_BATCH_TIMEOUT_MS);
let max_inflight = config
.lookup_max_inflight_requests
.unwrap_or(DEFAULT_LOOKUP_MAX_INFLIGHT_REQUESTS);
let max_retries = config
.lookup_max_retries
.unwrap_or(DEFAULT_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,
shutdown_tx,
}
}

/// 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>>> {
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(),
source: None,
})?;

// Wait for result
result_rx.await.map_err(|_| Error::UnexpectedError {
message: "Lookup result channel closed".to_string(),
source: None,
})?
}

/// Closes the lookup client gracefully.
///
/// This will:
/// 1. Stop accepting new lookups
/// 2. Process remaining lookups in the queue
/// 3. Wait for the sender to complete
pub async fn close(&self, timeout: Duration) {
debug!("Closing lookup client");

// Send shutdown signal
let _ = self.shutdown_tx.send(()).await;

// Wait for sender to complete with timeout
let _handle = &self.sender_handle;
if tokio::time::timeout(timeout, async {
// We can't actually await the handle here since we don't own it
// Just sleep to give it time to complete
tokio::time::sleep(timeout).await;
})
.await
.is_err()
{
error!("Lookup sender did not complete within timeout");
}

debug!("Lookup client closed");
}
}

impl Drop for LookupClient {
fn drop(&mut self) {
// Abort the sender task on drop
self.sender_handle.abort();
}
}
105 changes: 105 additions & 0 deletions crates/fluss/src/client/lookup/lookup_query.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,105 @@
// 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 query representation for batching lookup operations.

use crate::metadata::{TableBucket, TablePath};
use std::sync::atomic::{AtomicI32, Ordering};
use tokio::sync::oneshot;

/// Represents a single lookup query that will be batched and sent to the server.
///
/// Each `LookupQuery` contains:
/// - The table path and bucket for routing
/// - The encoded key bytes to look up
/// - A oneshot channel for returning the result
/// - A retry counter for handling transient failures
pub struct LookupQuery {
/// The table path for this lookup
#[allow(dead_code)]
table_path: TablePath,
/// The table bucket for this lookup
table_bucket: TableBucket,
/// The encoded primary key bytes
key: Vec<u8>,
/// Channel to send the result back to the caller
result_tx: Option<oneshot::Sender<Result<Option<Vec<u8>>, crate::error::Error>>>,
/// Number of retry attempts
retries: AtomicI32,
}

impl LookupQuery {
/// Creates a new lookup query.
pub fn new(
table_path: TablePath,
table_bucket: TableBucket,
key: Vec<u8>,
result_tx: oneshot::Sender<Result<Option<Vec<u8>>, crate::error::Error>>,
) -> Self {
Self {
table_path,
table_bucket,
key,
result_tx: Some(result_tx),
retries: AtomicI32::new(0),
}
}

/// Returns the table path.
#[allow(dead_code)]
pub fn table_path(&self) -> &TablePath {
&self.table_path
}

/// Returns the table bucket.
pub fn table_bucket(&self) -> &TableBucket {
&self.table_bucket
}

/// Returns the encoded key bytes.
#[allow(dead_code)]
pub fn key(&self) -> &[u8] {
&self.key
}

/// Takes ownership of the key bytes.
pub fn take_key(&mut self) -> Vec<u8> {
std::mem::take(&mut self.key)
}

/// Returns the current retry count.
pub fn retries(&self) -> i32 {
self.retries.load(Ordering::Acquire)
}

/// Increments the retry counter.
pub fn increment_retries(&self) {
self.retries.fetch_add(1, Ordering::AcqRel);
}

/// Completes the lookup with a result.
pub fn complete(&mut self, result: Result<Option<Vec<u8>>, crate::error::Error>) {
if let Some(tx) = self.result_tx.take() {
let _ = tx.send(result);
}
}

/// Returns true if the result has already been sent.
pub fn is_done(&self) -> bool {
self.result_tx.is_none()
}
}
Loading