17
17
18
18
use std:: collections:: HashMap ;
19
19
20
- use futures:: channel:: oneshot;
21
- use futures:: future:: join_all;
22
20
use futures:: { StreamExt , TryStreamExt } ;
23
21
use tokio:: sync:: oneshot:: { Receiver , channel} ;
24
22
25
- use super :: delete_filter:: { DeleteFilter , EqDelFuture } ;
23
+ use super :: delete_filter:: DeleteFilter ;
26
24
use crate :: arrow:: delete_file_loader:: BasicDeleteFileLoader ;
27
25
use crate :: delete_vector:: DeleteVector ;
28
26
use crate :: expr:: Predicate ;
@@ -40,18 +38,17 @@ pub(crate) struct CachingDeleteFileLoader {
40
38
// Intermediate context during processing of a delete file task.
41
39
enum DeleteFileContext {
42
40
// TODO: Delete Vector loader from Puffin files
43
- InProgEqDel ( EqDelFuture ) ,
41
+ ExistingEqDel ,
44
42
PosDels ( ArrowRecordBatchStream ) ,
45
43
FreshEqDel {
46
44
batch_stream : ArrowRecordBatchStream ,
47
- sender : oneshot:: Sender < Predicate > ,
45
+ sender : tokio :: sync :: oneshot:: Sender < Predicate > ,
48
46
} ,
49
47
}
50
48
51
49
// Final result of the processing of a delete file task before
52
50
// results are fully merged into the DeleteFileManager's state
53
51
enum ParsedDeleteFileContext {
54
- InProgEqDel ( EqDelFuture ) ,
55
52
DelVecs ( HashMap < String , DeleteVector > ) ,
56
53
EqDel ,
57
54
}
@@ -65,23 +62,23 @@ impl CachingDeleteFileLoader {
65
62
}
66
63
}
67
64
68
- /// Load the deletes for all the specified tasks
65
+ /// Initiates loading of all deletes for all the specified tasks
69
66
///
70
- /// Returned future completes once all loading has finished.
67
+ /// Returned future completes once all positional deletes and delete vectors
68
+ /// have loaded. EQ deletes are not waited for in this method but the returned
69
+ /// DeleteFilter will await their loading when queried for them.
71
70
///
72
71
/// * Create a single stream of all delete file tasks irrespective of type,
73
72
/// so that we can respect the combined concurrency limit
74
73
/// * We then process each in two phases: load and parse.
75
74
/// * for positional deletes the load phase instantiates an ArrowRecordBatchStream to
76
75
/// stream the file contents out
77
76
/// * for eq deletes, we first check if the EQ delete is already loaded or being loaded by
78
- /// another concurrently processing data file scan task. If it is, we return a future
79
- /// for the pre-existing task from the load phase. If not, we create such a future
80
- /// and store it in the state to prevent other data file tasks from starting to load
81
- /// the same equality delete file, and return a record batch stream from the load phase
82
- /// as per the other delete file types - only this time it is accompanied by a one-shot
83
- /// channel sender that we will eventually use to resolve the shared future that we stored
84
- /// in the state.
77
+ /// another concurrently processing data file scan task. If it is, we skip it.
78
+ /// If not, the DeleteFilter is updated to contain a notifier to prevent other data file
79
+ /// tasks from starting to load the same equality delete file. We spawn a task to load
80
+ /// the EQ delete's record batch stream, convert it to a predicate, update the delete filter,
81
+ /// and notify any task that was waiting for it.
85
82
/// * When this gets updated to add support for delete vectors, the load phase will return
86
83
/// a PuffinReader for them.
87
84
/// * The parse phase parses each record batch stream according to its associated data type.
@@ -100,35 +97,34 @@ impl CachingDeleteFileLoader {
100
97
/// ```none
101
98
/// FileScanTaskDeleteFile
102
99
/// |
103
- /// Already-loading EQ Delete | Everything Else
104
- /// +---------------------------------------------------+
105
- /// | |
106
- /// [get existing future] [load recordbatch stream / puffin]
107
- /// DeleteFileContext::InProgEqDel DeleteFileContext
108
- /// | |
109
- /// | |
110
- /// | +-----------------------------+--------------------------+
111
- /// | Pos Del Del Vec (Not yet Implemented) EQ Del
112
- /// | | | |
113
- /// | [parse pos del stream] [parse del vec puffin] [parse eq del]
114
- /// | HashMap<String, RoaringTreeMap> HashMap<String, RoaringTreeMap> (Predicate, Sender)
115
- /// | | | |
116
- /// | | | [persist to state]
117
- /// | | | ()
118
- /// | | | |
119
- /// | +-----------------------------+--------------------------+
120
- /// | |
121
- /// | [buffer unordered]
122
- /// | |
123
- /// | [combine del vectors]
124
- /// | HashMap<String, RoaringTreeMap>
125
- /// | |
126
- /// | [persist del vectors to state]
127
- /// | ()
128
- /// | |
129
- /// +-------------------------+-------------------------+
130
- /// |
131
- /// [join!]
100
+ /// Skip Started EQ Deletes
101
+ /// |
102
+ /// |
103
+ /// [load recordbatch stream / puffin]
104
+ /// DeleteFileContext
105
+ /// |
106
+ /// |
107
+ /// +-----------------------------+--------------------------+
108
+ /// Pos Del Del Vec (Not yet Implemented) EQ Del
109
+ /// | | |
110
+ /// [parse pos del stream] [parse del vec puffin] [parse eq del]
111
+ /// HashMap<String, RoaringTreeMap> HashMap<String, RoaringTreeMap> (Predicate, Sender)
112
+ /// | | |
113
+ /// | | [persist to state]
114
+ /// | | ()
115
+ /// | | |
116
+ /// +-----------------------------+--------------------------+
117
+ /// |
118
+ /// [buffer unordered]
119
+ /// |
120
+ /// [combine del vectors]
121
+ /// HashMap<String, RoaringTreeMap>
122
+ /// |
123
+ /// [persist del vectors to state]
124
+ /// ()
125
+ /// |
126
+ /// |
127
+ /// [join!]
132
128
/// ```
133
129
pub ( crate ) fn load_deletes (
134
130
& self ,
@@ -150,6 +146,7 @@ impl CachingDeleteFileLoader {
150
146
} )
151
147
. collect :: < Vec < _ > > ( ) ;
152
148
let task_stream = futures:: stream:: iter ( stream_items) ;
149
+
153
150
let del_filter = del_filter. clone ( ) ;
154
151
let concurrency_limit_data_files = self . concurrency_limit_data_files ;
155
152
let basic_delete_file_loader = self . basic_delete_file_loader . clone ( ) ;
@@ -178,16 +175,6 @@ impl CachingDeleteFileLoader {
178
175
. try_collect :: < Vec < _ > > ( )
179
176
. await ?;
180
177
181
- // wait for all in-progress EQ deletes from other tasks
182
- let _ = join_all ( results. iter ( ) . filter_map ( |i| {
183
- if let ParsedDeleteFileContext :: InProgEqDel ( fut) = i {
184
- Some ( fut. clone ( ) )
185
- } else {
186
- None
187
- }
188
- } ) )
189
- . await ;
190
-
191
178
for item in results {
192
179
if let ParsedDeleteFileContext :: DelVecs ( hash_map) = item {
193
180
for ( data_file_path, delete_vector) in hash_map. into_iter ( ) {
@@ -220,20 +207,13 @@ impl CachingDeleteFileLoader {
220
207
) ) ,
221
208
222
209
DataContentType :: EqualityDeletes => {
223
- let sender = {
224
- if let Some ( existing) = del_filter
225
- . get_equality_delete_predicate_for_delete_file_path ( & task. file_path )
226
- {
227
- return Ok ( DeleteFileContext :: InProgEqDel ( existing. clone ( ) ) ) ;
228
- }
229
-
230
- let ( sender, fut) = EqDelFuture :: new ( ) ;
231
-
232
- del_filter. insert_equality_delete ( task. file_path . to_string ( ) , fut) ;
233
-
234
- sender
210
+ let Some ( notify) = del_filter. try_start_eq_del_load ( & task. file_path ) else {
211
+ return Ok ( DeleteFileContext :: ExistingEqDel ) ;
235
212
} ;
236
213
214
+ let ( sender, receiver) = channel ( ) ;
215
+ del_filter. insert_equality_delete ( & task. file_path , receiver) ;
216
+
237
217
Ok ( DeleteFileContext :: FreshEqDel {
238
218
batch_stream : BasicDeleteFileLoader :: evolve_schema (
239
219
basic_delete_file_loader
@@ -257,7 +237,7 @@ impl CachingDeleteFileLoader {
257
237
ctx : DeleteFileContext ,
258
238
) -> Result < ParsedDeleteFileContext > {
259
239
match ctx {
260
- DeleteFileContext :: InProgEqDel ( fut ) => Ok ( ParsedDeleteFileContext :: InProgEqDel ( fut ) ) ,
240
+ DeleteFileContext :: ExistingEqDel => Ok ( ParsedDeleteFileContext :: EqDel ) ,
261
241
DeleteFileContext :: PosDels ( batch_stream) => {
262
242
let del_vecs =
263
243
Self :: parse_positional_deletes_record_batch_stream ( batch_stream) . await ?;
0 commit comments