@@ -118,16 +118,23 @@ impl FlightTableFactory {
118118 . map_err ( to_df_err) ?;
119119 let num_rows = precision ( metadata. info . total_records ) ;
120120 let total_byte_size = precision ( metadata. info . total_bytes ) ;
121- let logical_schema = metadata. schema ;
121+ let logical_schema = metadata. schema . clone ( ) ;
122122 let stats = Statistics {
123123 num_rows,
124124 total_byte_size,
125125 column_statistics : vec ! [ ] ,
126126 } ;
127+ let metadata_supplier = if metadata. props . reusable_flight_info {
128+ MetadataSupplier :: Reusable ( Arc :: new ( metadata) )
129+ } else {
130+ MetadataSupplier :: Refresh {
131+ driver : self . driver . clone ( ) ,
132+ channel,
133+ options,
134+ }
135+ } ;
127136 Ok ( FlightTable {
128- driver : self . driver . clone ( ) ,
129- channel,
130- options,
137+ metadata_supplier,
131138 origin,
132139 logical_schema,
133140 stats,
@@ -203,26 +210,41 @@ impl TryFrom<FlightInfo> for FlightMetadata {
203210/// for controlling the protocol and query execution details.
204211#[ derive( Clone , Debug , Default , Deserialize , Eq , PartialEq , Serialize ) ]
205212pub struct FlightProperties {
206- unbounded_stream : bool ,
213+ unbounded_streams : bool ,
207214 grpc_headers : HashMap < String , String > ,
208215 size_limits : SizeLimits ,
216+ reusable_flight_info : bool ,
209217}
210218
211219impl FlightProperties {
212- pub fn unbounded_stream ( mut self , unbounded_stream : bool ) -> Self {
213- self . unbounded_stream = unbounded_stream;
220+ pub fn new ( ) -> Self {
221+ Default :: default ( )
222+ }
223+
224+ /// Whether the service will produce infinite streams
225+ pub fn with_unbounded_streams ( mut self , unbounded_streams : bool ) -> Self {
226+ self . unbounded_streams = unbounded_streams;
214227 self
215228 }
216229
217- pub fn grpc_headers ( mut self , grpc_headers : HashMap < String , String > ) -> Self {
230+ /// GRPC headers that to use on subsequent calls.
231+ pub fn with_grpc_headers ( mut self , grpc_headers : HashMap < String , String > ) -> Self {
218232 self . grpc_headers = grpc_headers;
219233 self
220234 }
221235
222- pub fn size_limits ( mut self , size_limits : SizeLimits ) -> Self {
236+ /// Max sizes in bytes for encoded/decoded GRPC messages.
237+ pub fn with_size_limits ( mut self , size_limits : SizeLimits ) -> Self {
223238 self . size_limits = size_limits;
224239 self
225240 }
241+
242+ /// Whether the FlightInfo objects produced by the service can be used multiple times
243+ /// or need to be refreshed before every table scan.
244+ pub fn with_reusable_flight_info ( mut self , reusable_flight_info : bool ) -> Self {
245+ self . reusable_flight_info = reusable_flight_info;
246+ self
247+ }
226248}
227249
228250/// Message size limits to be passed to the underlying gRPC library.
@@ -248,12 +270,38 @@ impl Default for SizeLimits {
248270 }
249271}
250272
273+ #[ derive( Clone , Debug ) ]
274+ enum MetadataSupplier {
275+ Reusable ( Arc < FlightMetadata > ) ,
276+ Refresh {
277+ driver : Arc < dyn FlightDriver > ,
278+ channel : Channel ,
279+ options : HashMap < String , String > ,
280+ } ,
281+ }
282+
283+ impl MetadataSupplier {
284+ async fn flight_metadata ( & self ) -> datafusion:: common:: Result < Arc < FlightMetadata > > {
285+ match self {
286+ Self :: Reusable ( metadata) => Ok ( metadata. clone ( ) ) ,
287+ Self :: Refresh {
288+ driver,
289+ channel,
290+ options,
291+ } => Ok ( Arc :: new (
292+ driver
293+ . metadata ( channel. clone ( ) , options)
294+ . await
295+ . map_err ( to_df_err) ?,
296+ ) ) ,
297+ }
298+ }
299+ }
300+
251301/// Table provider that wraps a specific flight from an Arrow Flight service
252302#[ derive( Debug ) ]
253303pub struct FlightTable {
254- driver : Arc < dyn FlightDriver > ,
255- channel : Channel ,
256- options : HashMap < String , String > ,
304+ metadata_supplier : MetadataSupplier ,
257305 origin : String ,
258306 logical_schema : SchemaRef ,
259307 stats : Statistics ,
@@ -280,13 +328,9 @@ impl TableProvider for FlightTable {
280328 _filters : & [ Expr ] ,
281329 _limit : Option < usize > ,
282330 ) -> datafusion:: common:: Result < Arc < dyn ExecutionPlan > > {
283- let metadata = self
284- . driver
285- . metadata ( self . channel . clone ( ) , & self . options )
286- . await
287- . map_err ( to_df_err) ?;
331+ let metadata = self . metadata_supplier . flight_metadata ( ) . await ?;
288332 Ok ( Arc :: new ( FlightExec :: try_new (
289- metadata,
333+ metadata. as_ref ( ) ,
290334 projection,
291335 & self . origin ,
292336 ) ?) )
0 commit comments