@@ -122,15 +122,10 @@ pub struct Query {
122
122
}
123
123
124
124
impl Query {
125
- pub async fn execute (
126
- & self ,
127
- stream_name : String ,
128
- ) -> Result < ( Vec < RecordBatch > , Vec < String > ) , ExecuteError > {
129
- let time_partition = STREAM_INFO . get_time_partition ( & stream_name) ?;
130
-
131
- let df = QUERY_SESSION
132
- . execute_logical_plan ( self . final_logical_plan ( & time_partition) )
133
- . await ?;
125
+ pub async fn execute ( & self ) -> Result < ( Vec < RecordBatch > , Vec < String > ) , ExecuteError > {
126
+ let time_partitions = self . get_time_partitions ( ) ?;
127
+ let logical_plan = self . final_logical_plan ( time_partitions) ;
128
+ let df = QUERY_SESSION . execute_logical_plan ( logical_plan) . await ?;
134
129
135
130
let fields = df
136
131
. schema ( )
@@ -148,8 +143,21 @@ impl Query {
148
143
Ok ( ( results, fields) )
149
144
}
150
145
146
+ /// Get the time partitions for the streams mentioned in the query
147
+ fn get_time_partitions ( & self ) -> Result < HashMap < String , String > , ExecuteError > {
148
+ let mut time_partitions = HashMap :: default ( ) ;
149
+ for stream_name in self . stream_names . iter ( ) {
150
+ let Some ( time_partition) = STREAM_INFO . get_time_partition ( stream_name) ? else {
151
+ continue ;
152
+ } ;
153
+ time_partitions. insert ( stream_name. clone ( ) , time_partition) ;
154
+ }
155
+
156
+ Ok ( time_partitions)
157
+ }
158
+
151
159
/// return logical plan with all time filters applied through
152
- fn final_logical_plan ( & self , time_partition : & Option < String > ) -> LogicalPlan {
160
+ fn final_logical_plan ( & self , time_partitions : HashMap < String , String > ) -> LogicalPlan {
153
161
// see https://github.com/apache/arrow-datafusion/pull/8400
154
162
// this can be eliminated in later version of datafusion but with slight caveat
155
163
// transform cannot modify stringified plans by itself
@@ -161,7 +169,7 @@ impl Query {
161
169
plan. plan . as_ref ( ) . clone ( ) ,
162
170
self . time_range . start . naive_utc ( ) ,
163
171
self . time_range . end . naive_utc ( ) ,
164
- time_partition ,
172
+ & time_partitions ,
165
173
) ;
166
174
LogicalPlan :: Explain ( Explain {
167
175
verbose : plan. verbose ,
@@ -178,15 +186,16 @@ impl Query {
178
186
x,
179
187
self . time_range . start . naive_utc ( ) ,
180
188
self . time_range . end . naive_utc ( ) ,
181
- time_partition ,
189
+ & time_partitions ,
182
190
)
183
191
. data
184
192
}
185
193
}
186
194
}
187
195
188
- pub fn stream_names ( & self ) -> & [ String ] {
189
- & self . stream_names
196
+ // name of the main/first stream in the query
197
+ pub fn first_stream_name ( & self ) -> Option < & String > {
198
+ self . stream_names . first ( )
190
199
}
191
200
}
192
201
@@ -223,80 +232,65 @@ fn transform(
223
232
plan : LogicalPlan ,
224
233
start_time : NaiveDateTime ,
225
234
end_time : NaiveDateTime ,
226
- time_partition : & Option < String > ,
235
+ time_partitions : & HashMap < String , String > ,
227
236
) -> Transformed < LogicalPlan > {
228
- plan. transform ( & |plan| match plan {
229
- LogicalPlan :: TableScan ( table) => {
230
- let mut new_filters = vec ! [ ] ;
231
- if !table_contains_any_time_filters ( & table, time_partition) {
232
- let mut _start_time_filter: Expr ;
233
- let mut _end_time_filter: Expr ;
234
- match time_partition {
235
- Some ( time_partition) => {
236
- _start_time_filter =
237
- PartialTimeFilter :: Low ( std:: ops:: Bound :: Included ( start_time) )
238
- . binary_expr ( Expr :: Column ( Column :: new (
239
- Some ( table. table_name . to_owned ( ) ) ,
240
- time_partition. clone ( ) ,
241
- ) ) ) ;
242
- _end_time_filter =
243
- PartialTimeFilter :: High ( std:: ops:: Bound :: Excluded ( end_time) )
244
- . binary_expr ( Expr :: Column ( Column :: new (
245
- Some ( table. table_name . to_owned ( ) ) ,
246
- time_partition,
247
- ) ) ) ;
248
- }
249
- None => {
250
- _start_time_filter =
251
- PartialTimeFilter :: Low ( std:: ops:: Bound :: Included ( start_time) )
252
- . binary_expr ( Expr :: Column ( Column :: new (
253
- Some ( table. table_name . to_owned ( ) ) ,
254
- event:: DEFAULT_TIMESTAMP_KEY ,
255
- ) ) ) ;
256
- _end_time_filter =
257
- PartialTimeFilter :: High ( std:: ops:: Bound :: Excluded ( end_time) )
258
- . binary_expr ( Expr :: Column ( Column :: new (
259
- Some ( table. table_name . to_owned ( ) ) ,
260
- event:: DEFAULT_TIMESTAMP_KEY ,
261
- ) ) ) ;
262
- }
263
- }
264
-
265
- new_filters. push ( _start_time_filter) ;
266
- new_filters. push ( _end_time_filter) ;
267
- }
268
- let new_filter = new_filters. into_iter ( ) . reduce ( and) ;
269
- if let Some ( new_filter) = new_filter {
270
- let filter =
271
- Filter :: try_new ( new_filter, Arc :: new ( LogicalPlan :: TableScan ( table) ) ) . unwrap ( ) ;
272
- Ok ( Transformed :: yes ( LogicalPlan :: Filter ( filter) ) )
273
- } else {
274
- Ok ( Transformed :: no ( LogicalPlan :: TableScan ( table) ) )
275
- }
237
+ plan. transform ( |plan| {
238
+ let LogicalPlan :: TableScan ( table) = plan else {
239
+ return Ok ( Transformed :: no ( plan) ) ;
240
+ } ;
241
+
242
+ // Early return if filters already exist
243
+ if query_can_be_filtered_on_stream_time_partition ( & table, time_partitions) {
244
+ return Ok ( Transformed :: no ( LogicalPlan :: TableScan ( table) ) ) ;
276
245
}
277
- x => Ok ( Transformed :: no ( x) ) ,
246
+
247
+ let stream = table. table_name . clone ( ) ;
248
+ let time_partition = time_partitions
249
+ . get ( stream. table ( ) )
250
+ . map ( |x| x. as_str ( ) )
251
+ . unwrap_or ( event:: DEFAULT_TIMESTAMP_KEY ) ;
252
+
253
+ // Create column expression once
254
+ let column_expr = Expr :: Column ( Column :: new ( Some ( stream. clone ( ) ) , time_partition) ) ;
255
+
256
+ // Build filters
257
+ let low_filter = PartialTimeFilter :: Low ( std:: ops:: Bound :: Included ( start_time) )
258
+ . binary_expr ( column_expr. clone ( ) ) ;
259
+ let high_filter =
260
+ PartialTimeFilter :: High ( std:: ops:: Bound :: Excluded ( end_time) ) . binary_expr ( column_expr) ;
261
+
262
+ // Combine filters
263
+ let new_filter = and ( low_filter, high_filter) ;
264
+ let filter = Filter :: try_new ( new_filter, Arc :: new ( LogicalPlan :: TableScan ( table) ) ) . unwrap ( ) ;
265
+
266
+ Ok ( Transformed :: yes ( LogicalPlan :: Filter ( filter) ) )
278
267
} )
279
268
. expect ( "transform only transforms the tablescan" )
280
269
}
281
270
282
- fn table_contains_any_time_filters (
271
+ // check if the query contains the streams's time partition as filter
272
+ fn query_can_be_filtered_on_stream_time_partition (
283
273
table : & datafusion:: logical_expr:: TableScan ,
284
- time_partition : & Option < String > ,
274
+ time_partitions : & HashMap < String , String > ,
285
275
) -> bool {
286
276
table
287
277
. filters
288
278
. iter ( )
289
- . filter_map ( |x| {
290
- if let Expr :: BinaryExpr ( binexpr) = x {
291
- Some ( binexpr)
292
- } else {
293
- None
294
- }
279
+ . filter_map ( |x| match x {
280
+ Expr :: BinaryExpr ( binexpr) => Some ( binexpr) ,
281
+ _ => None ,
295
282
} )
296
- . any ( |expr| {
297
- matches ! ( & * expr. left, Expr :: Column ( Column { name, .. } )
298
- if ( ( time_partition. is_some( ) && name == time_partition. as_ref( ) . unwrap( ) ) ||
299
- ( !time_partition. is_some( ) && name == event:: DEFAULT_TIMESTAMP_KEY ) ) )
283
+ . any ( |expr| match & * expr. left {
284
+ Expr :: Column ( Column {
285
+ name : column_name, ..
286
+ } ) => {
287
+ time_partitions
288
+ . get ( table. table_name . table ( ) )
289
+ . map ( |x| x. as_str ( ) )
290
+ . unwrap_or ( event:: DEFAULT_TIMESTAMP_KEY )
291
+ == column_name
292
+ }
293
+ _ => false ,
300
294
} )
301
295
}
302
296
0 commit comments