@@ -2,13 +2,12 @@ use arrow::util::pretty::pretty_format_batches;
22use arrow_flight:: flight_service_client:: FlightServiceClient ;
33use async_trait:: async_trait;
44use datafusion:: common:: DataFusionError ;
5- use datafusion:: common:: utils:: get_available_parallelism;
65use datafusion:: execution:: SessionStateBuilder ;
7- use datafusion:: physical_plan:: displayable;
86use datafusion:: prelude:: { ParquetReadOptions , SessionContext } ;
97use datafusion_distributed:: {
108 ArrowFlightEndpoint , BoxCloneSyncChannel , ChannelResolver , DistributedExt ,
119 DistributedPhysicalOptimizerRule , DistributedSessionBuilderContext , create_flight_client,
10+ display_plan_ascii,
1211} ;
1312use futures:: TryStreamExt ;
1413use hyper_util:: rt:: TokioIo ;
@@ -20,20 +19,16 @@ use tonic::transport::{Endpoint, Server};
2019#[ derive( StructOpt ) ]
2120#[ structopt(
2221 name = "run" ,
23- about = "An in -memory cluster Distributed DataFusion runner "
22+ about = "Run a query in an in -memory Distributed DataFusion cluster "
2423) ]
2524struct Args {
25+ /// The SQL query to run.
2626 #[ structopt( ) ]
2727 query : String ,
2828
29+ /// Whether the distributed plan should be rendered instead of executing the query.
2930 #[ structopt( long) ]
30- explain : bool ,
31-
32- #[ structopt( long) ]
33- files_per_task : Option < usize > ,
34-
35- #[ structopt( long) ]
36- cardinality_task_sf : Option < f64 > ,
31+ show_distributed_plan : bool ,
3732}
3833
3934#[ tokio:: main]
@@ -44,31 +39,18 @@ async fn main() -> Result<(), Box<dyn Error>> {
4439 . with_default_features ( )
4540 . with_distributed_channel_resolver ( InMemoryChannelResolver :: new ( ) )
4641 . with_physical_optimizer_rule ( Arc :: new ( DistributedPhysicalOptimizerRule ) )
47- . with_distributed_files_per_task (
48- args. files_per_task . unwrap_or ( get_available_parallelism ( ) ) ,
49- ) ?
50- . with_distributed_cardinality_effect_task_scale_factor (
51- args. cardinality_task_sf . unwrap_or ( 1. ) ,
52- ) ?
42+ . with_distributed_files_per_task ( 1 ) ?
5343 . build ( ) ;
5444
5545 let ctx = SessionContext :: from ( state) ;
5646
57- ctx. register_parquet (
58- "flights_1m" ,
59- "testdata/flights-1m.parquet" ,
60- ParquetReadOptions :: default ( ) ,
61- )
62- . await ?;
63-
6447 ctx. register_parquet ( "weather" , "testdata/weather" , ParquetReadOptions :: default ( ) )
6548 . await ?;
6649
6750 let df = ctx. sql ( & args. query ) . await ?;
68- if args. explain {
51+ if args. show_distributed_plan {
6952 let plan = df. create_physical_plan ( ) . await ?;
70- let display = displayable ( plan. as_ref ( ) ) . indent ( true ) . to_string ( ) ;
71- println ! ( "{display}" ) ;
53+ println ! ( "{}" , display_plan_ascii( plan. as_ref( ) , false ) ) ;
7254 } else {
7355 let stream = df. execute_stream ( ) . await ?;
7456 let batches = stream. try_collect :: < Vec < _ > > ( ) . await ?;
@@ -133,7 +115,7 @@ impl InMemoryChannelResolver {
133115#[ async_trait]
134116impl ChannelResolver for InMemoryChannelResolver {
135117 fn get_urls ( & self ) -> Result < Vec < url:: Url > , DataFusionError > {
136- Ok ( vec ! [ url:: Url :: parse( DUMMY_URL ) . unwrap( ) ] )
118+ Ok ( vec ! [ url:: Url :: parse( DUMMY_URL ) . unwrap( ) ; 16 ] ) // simulate 16 workers.
137119 }
138120
139121 async fn get_flight_client_for_url (
0 commit comments