-
Notifications
You must be signed in to change notification settings - Fork 750
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Expose Inner FlightServiceClient on FlightSqlServiceClient (#3551) #3556
Merged
tustvold
merged 3 commits into
apache:master
from
tustvold:remove-unnecesary-mutex-flight-client
Jan 18, 2023
Merged
Changes from 2 commits
Commits
Show all changes
3 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -19,7 +19,6 @@ use base64::prelude::BASE64_STANDARD; | |
use base64::Engine; | ||
use bytes::Bytes; | ||
use std::collections::HashMap; | ||
use std::sync::Arc; | ||
use std::time::Duration; | ||
|
||
use crate::flight_service_client::FlightServiceClient; | ||
|
@@ -45,7 +44,6 @@ use arrow_ipc::{root_as_message, MessageHeader}; | |
use arrow_schema::{ArrowError, Schema, SchemaRef}; | ||
use futures::{stream, TryStreamExt}; | ||
use prost::Message; | ||
use tokio::sync::{Mutex, MutexGuard}; | ||
#[cfg(feature = "tls")] | ||
use tonic::transport::{Certificate, ClientTlsConfig, Identity}; | ||
use tonic::transport::{Channel, Endpoint}; | ||
|
@@ -56,7 +54,7 @@ use tonic::Streaming; | |
#[derive(Debug, Clone)] | ||
pub struct FlightSqlServiceClient { | ||
token: Option<String>, | ||
flight_client: Arc<Mutex<FlightServiceClient<Channel>>>, | ||
flight_client: FlightServiceClient<Channel>, | ||
} | ||
|
||
/// A FlightSql protocol client that can run queries against FlightSql servers | ||
|
@@ -124,16 +122,18 @@ impl FlightSqlServiceClient { | |
let flight_client = FlightServiceClient::new(channel); | ||
FlightSqlServiceClient { | ||
token: None, | ||
flight_client: Arc::new(Mutex::new(flight_client)), | ||
flight_client, | ||
} | ||
} | ||
|
||
fn mut_client( | ||
&mut self, | ||
) -> Result<MutexGuard<FlightServiceClient<Channel>>, ArrowError> { | ||
self.flight_client | ||
.try_lock() | ||
.map_err(|_| ArrowError::IoError("Unable to lock client".to_string())) | ||
/// Return a reference to the underlying [`FlightServiceClient`] | ||
pub fn inner(&self) -> &FlightServiceClient<Channel> { | ||
&self.flight_client | ||
} | ||
|
||
/// Return a mutable reference to the underlying [`FlightServiceClient`] | ||
pub fn inner_mut(&mut self) -> &mut FlightServiceClient<Channel> { | ||
&mut self.flight_client | ||
} | ||
|
||
async fn get_flight_info_for_command<M: ProstMessageExt>( | ||
|
@@ -142,7 +142,7 @@ impl FlightSqlServiceClient { | |
) -> Result<FlightInfo, ArrowError> { | ||
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec()); | ||
let fi = self | ||
.mut_client()? | ||
.flight_client | ||
.get_flight_info(descriptor) | ||
.await | ||
.map_err(status_to_arrow_error)? | ||
|
@@ -174,7 +174,7 @@ impl FlightSqlServiceClient { | |
.map_err(|_| ArrowError::ParseError("Cannot parse header".to_string()))?; | ||
req.metadata_mut().insert("authorization", val); | ||
let resp = self | ||
.mut_client()? | ||
.flight_client | ||
.handshake(req) | ||
.await | ||
.map_err(|e| ArrowError::IoError(format!("Can't handshake {}", e)))?; | ||
|
@@ -208,7 +208,7 @@ impl FlightSqlServiceClient { | |
let cmd = CommandStatementUpdate { query }; | ||
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec()); | ||
let mut result = self | ||
.mut_client()? | ||
.flight_client | ||
.do_put(stream::iter(vec![FlightData { | ||
flight_descriptor: Some(descriptor), | ||
..Default::default() | ||
|
@@ -247,7 +247,7 @@ impl FlightSqlServiceClient { | |
ticket: Ticket, | ||
) -> Result<Streaming<FlightData>, ArrowError> { | ||
Ok(self | ||
.mut_client()? | ||
.flight_client | ||
.do_get(ticket) | ||
.await | ||
.map_err(status_to_arrow_error)? | ||
|
@@ -332,7 +332,7 @@ impl FlightSqlServiceClient { | |
req.metadata_mut().insert("authorization", val); | ||
} | ||
let mut result = self | ||
.mut_client()? | ||
.flight_client | ||
.do_action(req) | ||
.await | ||
.map_err(status_to_arrow_error)? | ||
|
@@ -369,7 +369,7 @@ impl FlightSqlServiceClient { | |
/// A PreparedStatement | ||
#[derive(Debug, Clone)] | ||
pub struct PreparedStatement<T> { | ||
flight_client: Arc<Mutex<FlightServiceClient<T>>>, | ||
flight_client: FlightServiceClient<T>, | ||
parameter_binding: Option<RecordBatch>, | ||
handle: Bytes, | ||
dataset_schema: Schema, | ||
|
@@ -378,13 +378,13 @@ pub struct PreparedStatement<T> { | |
|
||
impl PreparedStatement<Channel> { | ||
pub(crate) fn new( | ||
client: Arc<Mutex<FlightServiceClient<Channel>>>, | ||
flight_client: FlightServiceClient<Channel>, | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Channel is cheaply cloneable, so there is no reason for the additional See https://docs.rs/tonic/latest/tonic/transport/struct.Channel.html#multiplexing-requests |
||
handle: impl Into<Bytes>, | ||
dataset_schema: Schema, | ||
parameter_schema: Schema, | ||
) -> Self { | ||
PreparedStatement { | ||
flight_client: client, | ||
flight_client, | ||
parameter_binding: None, | ||
handle: handle.into(), | ||
dataset_schema, | ||
|
@@ -399,7 +399,7 @@ impl PreparedStatement<Channel> { | |
}; | ||
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec()); | ||
let result = self | ||
.mut_client()? | ||
.flight_client | ||
.get_flight_info(descriptor) | ||
.await | ||
.map_err(status_to_arrow_error)? | ||
|
@@ -414,7 +414,7 @@ impl PreparedStatement<Channel> { | |
}; | ||
let descriptor = FlightDescriptor::new_cmd(cmd.as_any().encode_to_vec()); | ||
let mut result = self | ||
.mut_client()? | ||
.flight_client | ||
.do_put(stream::iter(vec![FlightData { | ||
flight_descriptor: Some(descriptor), | ||
..Default::default() | ||
|
@@ -463,20 +463,12 @@ impl PreparedStatement<Channel> { | |
body: cmd.as_any().encode_to_vec().into(), | ||
}; | ||
let _ = self | ||
.mut_client()? | ||
.flight_client | ||
.do_action(action) | ||
.await | ||
.map_err(status_to_arrow_error)?; | ||
Ok(()) | ||
} | ||
|
||
fn mut_client( | ||
&mut self, | ||
) -> Result<MutexGuard<FlightServiceClient<Channel>>, ArrowError> { | ||
self.flight_client | ||
.try_lock() | ||
.map_err(|_| ArrowError::IoError("Unable to lock client".to_string())) | ||
} | ||
} | ||
|
||
fn decode_error_to_arrow_error(err: prost::DecodeError) -> ArrowError { | ||
|
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I also recommend an
into_inner()
function if possible