@@ -34,7 +34,7 @@ use crate::metadata_columns::{RESERVED_FIELD_ID_UNDERSCORE_POS, row_pos_field};
3434use crate :: runtime:: spawn;
3535use crate :: scan:: ArrowRecordBatchStream ;
3636use crate :: scan:: incremental:: {
37- AppendedFileScanTask , IncrementalFileScanTask , IncrementalFileScanTaskStream ,
37+ AppendedFileScanTask , DeleteScanTask , IncrementalFileScanTaskStreams ,
3838} ;
3939use crate :: { Error , ErrorKind , Result } ;
4040
@@ -57,120 +57,6 @@ pub type CombinedIncrementalBatchRecordStream =
5757/// Stream type for obtaining a separate stream of appended and deleted record batches.
5858pub type UnzippedIncrementalBatchRecordStream = ( ArrowRecordBatchStream , ArrowRecordBatchStream ) ;
5959
60- impl StreamsInto < ArrowReader , CombinedIncrementalBatchRecordStream >
61- for IncrementalFileScanTaskStream
62- {
63- /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns a
64- /// stream of Arrow `RecordBatch`es containing the data from the files.
65- fn stream ( self , reader : ArrowReader ) -> Result < CombinedIncrementalBatchRecordStream > {
66- let ( appends, deletes) =
67- StreamsInto :: < ArrowReader , UnzippedIncrementalBatchRecordStream > :: stream ( self , reader) ?;
68-
69- let left = appends. map ( |res| res. map ( |batch| ( IncrementalBatchType :: Append , batch) ) ) ;
70- let right = deletes. map ( |res| res. map ( |batch| ( IncrementalBatchType :: Delete , batch) ) ) ;
71-
72- Ok ( Box :: pin ( select ( left, right) ) as CombinedIncrementalBatchRecordStream )
73- }
74- }
75-
76- impl StreamsInto < ArrowReader , UnzippedIncrementalBatchRecordStream >
77- for IncrementalFileScanTaskStream
78- {
79- /// Takes a stream of `IncrementalFileScanTasks` and reads all the files. Returns two
80- /// separate streams of Arrow `RecordBatch`es containing appended data and deleted records.
81- fn stream ( self , reader : ArrowReader ) -> Result < UnzippedIncrementalBatchRecordStream > {
82- let ( appends_tx, appends_rx) =
83- channel :: < Result < RecordBatch > > ( reader. concurrency_limit_data_files ) ;
84- let ( deletes_tx, deletes_rx) =
85- channel :: < Result < RecordBatch > > ( reader. concurrency_limit_data_files ) ;
86-
87- let batch_size = reader. batch_size ;
88-
89- spawn ( async move {
90- let _ = self
91- . try_for_each_concurrent ( reader. concurrency_limit_data_files , |task| {
92- let file_io = reader. file_io . clone ( ) ;
93- match task {
94- IncrementalFileScanTask :: Append ( append_task) => {
95- let appends_tx = appends_tx. clone ( ) ;
96- Box :: pin ( async move {
97- spawn ( async move {
98- let record_batch_stream = process_incremental_append_task (
99- append_task,
100- batch_size,
101- file_io,
102- )
103- . await ;
104-
105- process_record_batch_stream (
106- record_batch_stream,
107- appends_tx,
108- "failed to read appended record batch" ,
109- )
110- . await ;
111- } ) ;
112- Ok ( ( ) )
113- } )
114- as Pin < Box < dyn futures:: Future < Output = Result < ( ) > > + Send > >
115- }
116- IncrementalFileScanTask :: Delete ( deleted_file_task) => {
117- let deletes_tx = deletes_tx. clone ( ) ;
118- Box :: pin ( async move {
119- spawn ( async move {
120- let file_path = deleted_file_task. data_file_path ( ) . to_string ( ) ;
121- let total_records =
122- deleted_file_task. base . record_count . unwrap_or ( 0 ) ;
123-
124- let record_batch_stream = process_incremental_deleted_file_task (
125- file_path,
126- total_records,
127- batch_size,
128- ) ;
129-
130- process_record_batch_stream (
131- record_batch_stream,
132- deletes_tx,
133- "failed to read deleted file record batch" ,
134- )
135- . await ;
136- } ) ;
137- Ok ( ( ) )
138- } )
139- as Pin < Box < dyn futures:: Future < Output = Result < ( ) > > + Send > >
140- }
141- IncrementalFileScanTask :: PositionalDeletes ( file_path, delete_vector) => {
142- let deletes_tx = deletes_tx. clone ( ) ;
143- Box :: pin ( async move {
144- spawn ( async move {
145- let record_batch_stream = process_incremental_delete_task (
146- file_path,
147- delete_vector,
148- batch_size,
149- ) ;
150-
151- process_record_batch_stream (
152- record_batch_stream,
153- deletes_tx,
154- "failed to read deleted record batch" ,
155- )
156- . await ;
157- } ) ;
158- Ok ( ( ) )
159- } )
160- as Pin < Box < dyn futures:: Future < Output = Result < ( ) > > + Send > >
161- }
162- }
163- } )
164- . await ;
165- } ) ;
166-
167- Ok ( (
168- Box :: pin ( appends_rx) as ArrowRecordBatchStream ,
169- Box :: pin ( deletes_rx) as ArrowRecordBatchStream ,
170- ) )
171- }
172- }
173-
17460async fn process_incremental_append_task (
17561 task : AppendedFileScanTask ,
17662 batch_size : Option < usize > ,
@@ -330,3 +216,117 @@ fn process_incremental_deleted_file_task(
330216
331217 Ok ( Box :: pin ( stream) as ArrowRecordBatchStream )
332218}
219+
220+ impl StreamsInto < ArrowReader , CombinedIncrementalBatchRecordStream >
221+ for IncrementalFileScanTaskStreams
222+ {
223+ /// Takes separate streams of appended and deleted file scan tasks and reads all the files.
224+ /// Returns a combined stream of Arrow `RecordBatch`es containing the data from the files.
225+ fn stream ( self , reader : ArrowReader ) -> Result < CombinedIncrementalBatchRecordStream > {
226+ let ( appends, deletes) =
227+ StreamsInto :: < ArrowReader , UnzippedIncrementalBatchRecordStream > :: stream ( self , reader) ?;
228+
229+ let left = appends. map ( |res| res. map ( |batch| ( IncrementalBatchType :: Append , batch) ) ) ;
230+ let right = deletes. map ( |res| res. map ( |batch| ( IncrementalBatchType :: Delete , batch) ) ) ;
231+
232+ Ok ( Box :: pin ( select ( left, right) ) as CombinedIncrementalBatchRecordStream )
233+ }
234+ }
235+
236+ impl StreamsInto < ArrowReader , UnzippedIncrementalBatchRecordStream >
237+ for IncrementalFileScanTaskStreams
238+ {
239+ /// Takes separate streams of appended and deleted file scan tasks and reads all the files.
240+ /// Returns two separate streams of Arrow `RecordBatch`es containing appended data and deleted records.
241+ fn stream ( self , reader : ArrowReader ) -> Result < UnzippedIncrementalBatchRecordStream > {
242+ let ( appends_tx, appends_rx) =
243+ channel :: < Result < RecordBatch > > ( reader. concurrency_limit_data_files ) ;
244+ let ( deletes_tx, deletes_rx) =
245+ channel :: < Result < RecordBatch > > ( reader. concurrency_limit_data_files ) ;
246+
247+ let batch_size = reader. batch_size ;
248+
249+ let ( append_stream, delete_stream) = self ;
250+
251+ // Process append tasks
252+ let file_io = reader. file_io . clone ( ) ;
253+ spawn ( async move {
254+ let _ = append_stream
255+ . try_for_each_concurrent ( reader. concurrency_limit_data_files , |append_task| {
256+ let file_io = file_io. clone ( ) ;
257+ let appends_tx = appends_tx. clone ( ) ;
258+ async move {
259+ spawn ( async move {
260+ let record_batch_stream =
261+ process_incremental_append_task ( append_task, batch_size, file_io)
262+ . await ;
263+
264+ process_record_batch_stream (
265+ record_batch_stream,
266+ appends_tx,
267+ "failed to read appended record batch" ,
268+ )
269+ . await ;
270+ } ) ;
271+ Ok ( ( ) )
272+ }
273+ } )
274+ . await ;
275+ } ) ;
276+
277+ // Process delete tasks
278+ spawn ( async move {
279+ let _ = delete_stream
280+ . try_for_each_concurrent ( reader. concurrency_limit_data_files , |delete_task| {
281+ let deletes_tx = deletes_tx. clone ( ) ;
282+ async move {
283+ match delete_task {
284+ DeleteScanTask :: DeletedFile ( deleted_file_task) => {
285+ spawn ( async move {
286+ let file_path = deleted_file_task. data_file_path ( ) . to_string ( ) ;
287+ let total_records =
288+ deleted_file_task. base . record_count . unwrap_or ( 0 ) ;
289+
290+ let record_batch_stream = process_incremental_deleted_file_task (
291+ file_path,
292+ total_records,
293+ batch_size,
294+ ) ;
295+
296+ process_record_batch_stream (
297+ record_batch_stream,
298+ deletes_tx,
299+ "failed to read deleted file record batch" ,
300+ )
301+ . await ;
302+ } ) ;
303+ }
304+ DeleteScanTask :: PositionalDeletes ( file_path, delete_vector) => {
305+ spawn ( async move {
306+ let record_batch_stream = process_incremental_delete_task (
307+ file_path,
308+ delete_vector,
309+ batch_size,
310+ ) ;
311+
312+ process_record_batch_stream (
313+ record_batch_stream,
314+ deletes_tx,
315+ "failed to read deleted record batch" ,
316+ )
317+ . await ;
318+ } ) ;
319+ }
320+ }
321+ Ok ( ( ) )
322+ }
323+ } )
324+ . await ;
325+ } ) ;
326+
327+ Ok ( (
328+ Box :: pin ( appends_rx) as ArrowRecordBatchStream ,
329+ Box :: pin ( deletes_rx) as ArrowRecordBatchStream ,
330+ ) )
331+ }
332+ }
0 commit comments