@@ -93,6 +93,7 @@ use std::sync::Arc;
9393use std:: time:: SystemTime ;
9494use tracing:: { instrument, Instrument } ;
9595
96+ use super :: serialized_plan:: PreSerializedPlan ;
9697use super :: udfs:: {
9798 aggregate_udf_by_kind, registerable_aggregate_udfs, registerable_arc_aggregate_udfs,
9899 registerable_arc_scalar_udfs, CubeAggregateUDFKind ,
@@ -287,19 +288,19 @@ impl QueryExecutor for QueryExecutorImpl {
287288 plan : SerializedPlan ,
288289 cluster : Arc < dyn Cluster > ,
289290 ) -> Result < ( Arc < dyn ExecutionPlan > , LogicalPlan ) , CubeError > {
290- let plan_to_move = plan. logical_plan (
291+ let pre_serialized_plan = plan. to_pre_serialized (
291292 HashMap :: new ( ) ,
292293 HashMap :: new ( ) ,
293294 NoopParquetMetadataCache :: new ( ) ,
294295 ) ?;
295- let serialized_plan = Arc :: new ( plan ) ;
296- let ctx = self . router_context ( cluster. clone ( ) , serialized_plan . clone ( ) ) ?;
296+ let pre_serialized_plan = Arc :: new ( pre_serialized_plan ) ;
297+ let ctx = self . router_context ( cluster. clone ( ) , pre_serialized_plan . clone ( ) ) ?;
297298 Ok ( (
298299 ctx. clone ( )
299300 . state ( )
300- . create_physical_plan ( & plan_to_move . clone ( ) )
301+ . create_physical_plan ( pre_serialized_plan . logical_plan ( ) )
301302 . await ?,
302- plan_to_move ,
303+ pre_serialized_plan . logical_plan ( ) . clone ( ) ,
303304 ) )
304305 }
305306
@@ -310,20 +311,20 @@ impl QueryExecutor for QueryExecutorImpl {
310311 chunk_id_to_record_batches : HashMap < u64 , Vec < RecordBatch > > ,
311312 data_loaded_size : Option < Arc < DataLoadedSize > > ,
312313 ) -> Result < ( Arc < dyn ExecutionPlan > , LogicalPlan ) , CubeError > {
313- let plan_to_move = plan. logical_plan (
314+ let pre_serialized_plan = plan. to_pre_serialized (
314315 remote_to_local_names,
315316 chunk_id_to_record_batches,
316317 self . parquet_metadata_cache . cache ( ) . clone ( ) ,
317318 ) ?;
318- let plan = Arc :: new ( plan ) ;
319- let ctx = self . worker_context ( plan . clone ( ) , data_loaded_size) ?;
319+ let pre_serialized_plan = Arc :: new ( pre_serialized_plan ) ;
320+ let ctx = self . worker_context ( pre_serialized_plan . clone ( ) , data_loaded_size) ?;
320321 let plan_ctx = ctx. clone ( ) ;
321322 Ok ( (
322323 plan_ctx
323324 . state ( )
324- . create_physical_plan ( & plan_to_move . clone ( ) )
325+ . create_physical_plan ( pre_serialized_plan . logical_plan ( ) )
325326 . await ?,
326- plan_to_move ,
327+ pre_serialized_plan . logical_plan ( ) . clone ( ) ,
327328 ) )
328329 }
329330
@@ -372,7 +373,7 @@ impl QueryExecutorImpl {
372373 fn router_context (
373374 & self ,
374375 cluster : Arc < dyn Cluster > ,
375- serialized_plan : Arc < SerializedPlan > ,
376+ serialized_plan : Arc < PreSerializedPlan > ,
376377 ) -> Result < Arc < SessionContext > , CubeError > {
377378 let runtime = Arc :: new ( RuntimeEnv :: default ( ) ) ;
378379 let config = Self :: session_config ( ) ;
@@ -424,7 +425,7 @@ impl QueryExecutorImpl {
424425
425426 fn worker_context (
426427 & self ,
427- serialized_plan : Arc < SerializedPlan > ,
428+ serialized_plan : Arc < PreSerializedPlan > ,
428429 data_loaded_size : Option < Arc < DataLoadedSize > > ,
429430 ) -> Result < Arc < SessionContext > , CubeError > {
430431 let runtime = Arc :: new ( RuntimeEnv :: default ( ) ) ;
@@ -1229,7 +1230,7 @@ pub struct ClusterSendExec {
12291230 /// Never executed, only stored to allow consistent optimization on router and worker.
12301231 pub input_for_optimizations : Arc < dyn ExecutionPlan > ,
12311232 pub cluster : Arc < dyn Cluster > ,
1232- pub serialized_plan : Arc < SerializedPlan > ,
1233+ pub serialized_plan : Arc < PreSerializedPlan > ,
12331234 pub use_streaming : bool ,
12341235}
12351236
@@ -1248,7 +1249,7 @@ pub enum InlineCompoundPartition {
12481249impl ClusterSendExec {
12491250 pub fn new (
12501251 cluster : Arc < dyn Cluster > ,
1251- serialized_plan : Arc < SerializedPlan > ,
1252+ serialized_plan : Arc < PreSerializedPlan > ,
12521253 union_snapshots : & [ Snapshots ] ,
12531254 input_for_optimizations : Arc < dyn ExecutionPlan > ,
12541255 use_streaming : bool ,
@@ -1503,7 +1504,7 @@ impl ClusterSendExec {
15031504 }
15041505 }
15051506
1506- pub fn worker_plans ( & self ) -> Vec < ( String , SerializedPlan ) > {
1507+ pub fn worker_plans ( & self ) -> Vec < ( String , PreSerializedPlan ) > {
15071508 let mut res = Vec :: new ( ) ;
15081509 for ( node_name, partitions) in self . partitions . iter ( ) {
15091510 res. push ( (
@@ -1517,7 +1518,7 @@ impl ClusterSendExec {
15171518 fn serialized_plan_for_partitions (
15181519 & self ,
15191520 partitions : & ( Vec < ( u64 , RowRange ) > , Vec < InlineTableId > ) ,
1520- ) -> SerializedPlan {
1521+ ) -> PreSerializedPlan {
15211522 let ( partitions, inline_table_ids) = partitions;
15221523 let mut ps = HashMap :: < _ , RowFilter > :: new ( ) ;
15231524 for ( id, range) in partitions {
@@ -1583,13 +1584,13 @@ impl ExecutionPlan for ClusterSendExec {
15831584 let node_name = node_name. to_string ( ) ;
15841585 if self . use_streaming {
15851586 // A future that yields a stream
1586- let fut = async move { cluster. run_select_stream ( & node_name, plan) . await } ;
1587+ let fut = async move { cluster. run_select_stream ( & node_name, plan. to_serialized_plan ( ) ? ) . await } ;
15871588 // Use TryStreamExt::try_flatten to flatten the stream of streams
15881589 let stream = futures:: stream:: once ( fut) . try_flatten ( ) ;
15891590
15901591 Ok ( Box :: pin ( RecordBatchStreamAdapter :: new ( schema, stream) ) )
15911592 } else {
1592- let record_batches = async move { cluster. run_select ( & node_name, plan) . await } ;
1593+ let record_batches = async move { cluster. run_select ( & node_name, plan. to_serialized_plan ( ) ? ) . await } ;
15931594 let stream = futures:: stream:: once ( record_batches) . flat_map ( |r| match r {
15941595 Ok ( vec) => stream:: iter ( vec. into_iter ( ) . map ( |b| Ok ( b) ) . collect :: < Vec < _ > > ( ) ) ,
15951596 Err ( e) => stream:: iter ( vec ! [ Err ( DataFusionError :: Execution ( e. to_string( ) ) ) ] ) ,
0 commit comments