@@ -109,29 +109,32 @@ impl RunOpt {
109109 } ;
110110
111111 let mut benchmark_run = BenchmarkRun :: new ( ) ;
112- for query_id in query_range {
113- benchmark_run. start_new_case ( & format ! ( "Query {query_id}" ) ) ;
114- let query_run = self . benchmark_query ( query_id) . await ?;
115- for iter in query_run {
116- benchmark_run. write_iter ( iter. elapsed , iter. row_count ) ;
117- }
118- }
119- benchmark_run. maybe_write_json ( self . output_path . as_ref ( ) ) ?;
120- Ok ( ( ) )
121- }
122-
123- async fn benchmark_query ( & self , query_id : usize ) -> Result < Vec < QueryResult > > {
124112 let mut config = self
125113 . common
126114 . config ( ) ?
127115 . with_collect_statistics ( !self . disable_statistics ) ;
128116 config. options_mut ( ) . optimizer . prefer_hash_join = self . prefer_hash_join ;
129117 let rt_builder = self . common . runtime_env_builder ( ) ?;
130118 let ctx = SessionContext :: new_with_config_rt ( config, rt_builder. build_arc ( ) ?) ;
131-
132119 // register tables
133120 self . register_tables ( & ctx) . await ?;
134121
122+ for query_id in query_range {
123+ benchmark_run. start_new_case ( & format ! ( "Query {query_id}" ) ) ;
124+ let query_run = self . benchmark_query ( query_id, & ctx) . await ?;
125+ for iter in query_run {
126+ benchmark_run. write_iter ( iter. elapsed , iter. row_count ) ;
127+ }
128+ }
129+ benchmark_run. maybe_write_json ( self . output_path . as_ref ( ) ) ?;
130+ Ok ( ( ) )
131+ }
132+
133+ async fn benchmark_query (
134+ & self ,
135+ query_id : usize ,
136+ ctx : & SessionContext ,
137+ ) -> Result < Vec < QueryResult > > {
135138 let mut millis = vec ! [ ] ;
136139 // run benchmark
137140 let mut query_results = vec ! [ ] ;
@@ -146,14 +149,14 @@ impl RunOpt {
146149 if query_id == 15 {
147150 for ( n, query) in sql. iter ( ) . enumerate ( ) {
148151 if n == 1 {
149- result = self . execute_query ( & ctx, query) . await ?;
152+ result = self . execute_query ( ctx, query) . await ?;
150153 } else {
151- self . execute_query ( & ctx, query) . await ?;
154+ self . execute_query ( ctx, query) . await ?;
152155 }
153156 }
154157 } else {
155158 for query in sql {
156- result = self . execute_query ( & ctx, query) . await ?;
159+ result = self . execute_query ( ctx, query) . await ?;
157160 }
158161 }
159162
0 commit comments