1use std::sync::Arc;
5
6use arrow_array::{make_array, BooleanArray, RecordBatch, RecordBatchOptions, UInt64Array};
7use arrow_buffer::NullBuffer;
8use futures::{
9 future::BoxFuture,
10 stream::{BoxStream, FuturesOrdered},
11 FutureExt, Stream, StreamExt,
12};
13use lance_arrow::RecordBatchExt;
14use lance_core::{
15 utils::{address::RowAddress, deletion::DeletionVector},
16 Result, ROW_ADDR, ROW_ADDR_FIELD, ROW_ID, ROW_ID_FIELD,
17};
18use lance_io::ReadBatchParams;
19use tracing::{instrument, Instrument};
20
21use crate::rowids::RowIdSequence;
22
23pub type ReadBatchFut = BoxFuture<'static, Result<RecordBatch>>;
24pub struct ReadBatchTask {
27 pub task: ReadBatchFut,
28 pub num_rows: u32,
29}
30pub type ReadBatchTaskStream = BoxStream<'static, ReadBatchTask>;
31pub type ReadBatchFutStream = BoxStream<'static, ReadBatchFut>;
32
33struct MergeStream {
34 streams: Vec<ReadBatchTaskStream>,
35 next_batch: FuturesOrdered<ReadBatchFut>,
36 next_num_rows: u32,
37 index: usize,
38}
39
40impl MergeStream {
41 fn emit(&mut self) -> ReadBatchTask {
42 let mut iter = std::mem::take(&mut self.next_batch);
43 let task = async move {
44 let mut batch = iter.next().await.unwrap()?;
45 while let Some(next) = iter.next().await {
46 let next = next?;
47 batch = batch.merge(&next)?;
48 }
49 Ok(batch)
50 }
51 .boxed();
52 let num_rows = self.next_num_rows;
53 self.next_num_rows = 0;
54 ReadBatchTask { task, num_rows }
55 }
56}
57
58impl Stream for MergeStream {
59 type Item = ReadBatchTask;
60
61 fn poll_next(
62 mut self: std::pin::Pin<&mut Self>,
63 cx: &mut std::task::Context<'_>,
64 ) -> std::task::Poll<Option<Self::Item>> {
65 loop {
66 let index = self.index;
67 match self.streams[index].poll_next_unpin(cx) {
68 std::task::Poll::Ready(Some(batch_task)) => {
69 if self.index == 0 {
70 self.next_num_rows = batch_task.num_rows;
71 } else {
72 debug_assert_eq!(self.next_num_rows, batch_task.num_rows);
73 }
74 self.next_batch.push_back(batch_task.task);
75 self.index += 1;
76 if self.index == self.streams.len() {
77 self.index = 0;
78 let next_batch = self.emit();
79 return std::task::Poll::Ready(Some(next_batch));
80 }
81 }
82 std::task::Poll::Ready(None) => {
83 return std::task::Poll::Ready(None);
84 }
85 std::task::Poll::Pending => {
86 return std::task::Poll::Pending;
87 }
88 }
89 }
90 }
91}
92
93pub fn merge_streams(streams: Vec<ReadBatchTaskStream>) -> ReadBatchTaskStream {
106 MergeStream {
107 streams,
108 next_batch: FuturesOrdered::new(),
109 next_num_rows: 0,
110 index: 0,
111 }
112 .boxed()
113}
114
115fn apply_deletions_as_nulls(batch: RecordBatch, mask: &BooleanArray) -> Result<RecordBatch> {
122 let mask_buffer = NullBuffer::new(mask.values().clone());
126
127 match mask_buffer.null_count() {
128 0 => return Ok(batch),
130 _ => {}
131 }
132
133 let new_columns = batch
135 .schema()
136 .fields()
137 .iter()
138 .zip(batch.columns())
139 .map(|(field, col)| {
140 if field.name() == ROW_ID || field.name() == ROW_ADDR {
141 let col_data = col.to_data();
142 let null_buffer = NullBuffer::union(col_data.nulls(), Some(&mask_buffer));
145
146 Ok(col_data
147 .into_builder()
148 .null_bit_buffer(null_buffer.map(|b| b.buffer().clone()))
149 .build()
150 .map(make_array)?)
151 } else {
152 Ok(col.clone())
153 }
154 })
155 .collect::<Result<Vec<_>>>()?;
156
157 Ok(RecordBatch::try_new_with_options(
158 batch.schema(),
159 new_columns,
160 &RecordBatchOptions::new().with_row_count(Some(batch.num_rows())),
161 )?)
162}
163
164#[derive(Debug)]
166pub struct RowIdAndDeletesConfig {
167 pub params: ReadBatchParams,
169 pub with_row_id: bool,
171 pub with_row_addr: bool,
173 pub deletion_vector: Option<Arc<DeletionVector>>,
175 pub row_id_sequence: Option<Arc<RowIdSequence>>,
177 pub make_deletions_null: bool,
179 pub total_num_rows: u32,
183}
184
185#[instrument(level = "debug", skip_all)]
186pub fn apply_row_id_and_deletes(
187 batch: RecordBatch,
188 batch_offset: u32,
189 fragment_id: u32,
190 config: &RowIdAndDeletesConfig,
191) -> Result<RecordBatch> {
192 let mut deletion_vector = config.deletion_vector.as_ref();
193 if let Some(deletion_vector_inner) = deletion_vector {
195 if matches!(deletion_vector_inner.as_ref(), DeletionVector::NoDeletions) {
196 deletion_vector = None;
197 }
198 }
199 let has_deletions = deletion_vector.is_some();
200 debug_assert!(
201 batch.num_columns() > 0 || config.with_row_id || config.with_row_addr || has_deletions
202 );
203
204 let should_fetch_row_addr = config.with_row_addr
206 || (config.with_row_id && config.row_id_sequence.is_none())
207 || has_deletions;
208
209 let num_rows = batch.num_rows() as u32;
210
211 let row_addrs =
212 if should_fetch_row_addr {
213 let _rowaddrs = tracing::span!(tracing::Level::DEBUG, "fetch_row_addrs").entered();
214 let mut row_addrs = Vec::with_capacity(num_rows as usize);
215 for offset_range in config
216 .params
217 .slice(batch_offset as usize, num_rows as usize)
218 .unwrap()
219 .iter_offset_ranges()?
220 {
221 row_addrs.extend(offset_range.map(|row_offset| {
222 u64::from(RowAddress::new_from_parts(fragment_id, row_offset))
223 }));
224 }
225
226 Some(Arc::new(UInt64Array::from(row_addrs)))
227 } else {
228 None
229 };
230
231 let row_ids = if config.with_row_id {
232 let _rowids = tracing::span!(tracing::Level::DEBUG, "fetch_row_ids").entered();
233 if let Some(row_id_sequence) = &config.row_id_sequence {
234 let selection = config
235 .params
236 .slice(batch_offset as usize, num_rows as usize)
237 .unwrap()
238 .to_ranges()
239 .unwrap();
240 let row_ids = row_id_sequence
241 .select(
242 selection
243 .iter()
244 .flat_map(|r| r.start as usize..r.end as usize),
245 )
246 .collect::<UInt64Array>();
247 Some(Arc::new(row_ids))
248 } else {
249 row_addrs.clone()
252 }
253 } else {
254 None
255 };
256
257 let span = tracing::span!(tracing::Level::DEBUG, "apply_deletions");
258 let _enter = span.enter();
259 let deletion_mask = deletion_vector.and_then(|v| {
260 let row_addrs: &[u64] = row_addrs.as_ref().unwrap().values();
261 v.build_predicate(row_addrs.iter())
262 });
263
264 let batch = if config.with_row_id {
265 let row_id_arr = row_ids.unwrap();
266 batch.try_with_column(ROW_ID_FIELD.clone(), row_id_arr)?
267 } else {
268 batch
269 };
270
271 let batch = if config.with_row_addr {
272 let row_addr_arr = row_addrs.unwrap();
273 batch.try_with_column(ROW_ADDR_FIELD.clone(), row_addr_arr)?
274 } else {
275 batch
276 };
277
278 match (deletion_mask, config.make_deletions_null) {
279 (None, _) => Ok(batch),
280 (Some(mask), false) => Ok(arrow::compute::filter_record_batch(&batch, &mask)?),
281 (Some(mask), true) => Ok(apply_deletions_as_nulls(batch, &mask)?),
282 }
283}
284
285pub fn wrap_with_row_id_and_delete(
291 stream: ReadBatchTaskStream,
292 fragment_id: u32,
293 config: RowIdAndDeletesConfig,
294) -> ReadBatchFutStream {
295 let config = Arc::new(config);
296 let mut offset = 0;
297 stream
298 .map(move |batch_task| {
299 let config = config.clone();
300 let this_offset = offset;
301 let num_rows = batch_task.num_rows;
302 offset += num_rows;
303 let task = batch_task.task;
304 tokio::spawn(
305 async move {
306 let batch = task.await?;
307 apply_row_id_and_deletes(batch, this_offset, fragment_id, config.as_ref())
308 }
309 .in_current_span(),
310 )
311 .map(|join_wrapper| join_wrapper.unwrap())
312 .boxed()
313 })
314 .boxed()
315}
316
317#[cfg(test)]
318mod tests {
319 use std::sync::Arc;
320
321 use arrow::{array::AsArray, datatypes::UInt64Type};
322 use arrow_array::{types::Int32Type, RecordBatch, UInt32Array};
323 use arrow_schema::ArrowError;
324 use futures::{stream::BoxStream, FutureExt, StreamExt, TryStreamExt};
325 use lance_core::{
326 utils::{address::RowAddress, deletion::DeletionVector},
327 ROW_ID,
328 };
329 use lance_datagen::{BatchCount, RowCount};
330 use lance_io::{stream::arrow_stream_to_lance_stream, ReadBatchParams};
331 use roaring::RoaringBitmap;
332
333 use crate::utils::stream::ReadBatchTask;
334
335 use super::RowIdAndDeletesConfig;
336
337 fn batch_task_stream(
338 datagen_stream: BoxStream<'static, std::result::Result<RecordBatch, ArrowError>>,
339 ) -> super::ReadBatchTaskStream {
340 arrow_stream_to_lance_stream(datagen_stream)
341 .map(|batch| ReadBatchTask {
342 num_rows: batch.as_ref().unwrap().num_rows() as u32,
343 task: std::future::ready(batch).boxed(),
344 })
345 .boxed()
346 }
347
348 #[tokio::test]
349 async fn test_basic_zip() {
350 let left = batch_task_stream(
351 lance_datagen::gen()
352 .col("x", lance_datagen::array::step::<Int32Type>())
353 .into_reader_stream(RowCount::from(100), BatchCount::from(10))
354 .0,
355 );
356 let right = batch_task_stream(
357 lance_datagen::gen()
358 .col("y", lance_datagen::array::step::<Int32Type>())
359 .into_reader_stream(RowCount::from(100), BatchCount::from(10))
360 .0,
361 );
362
363 let merged = super::merge_streams(vec![left, right])
364 .map(|batch_task| batch_task.task)
365 .buffered(1)
366 .try_collect::<Vec<_>>()
367 .await
368 .unwrap();
369
370 let expected = lance_datagen::gen()
371 .col("x", lance_datagen::array::step::<Int32Type>())
372 .col("y", lance_datagen::array::step::<Int32Type>())
373 .into_reader_rows(RowCount::from(100), BatchCount::from(10))
374 .collect::<Result<Vec<_>, ArrowError>>()
375 .unwrap();
376 assert_eq!(merged, expected);
377 }
378
379 async fn check_row_id(params: ReadBatchParams, expected: impl IntoIterator<Item = u32>) {
380 let expected = Vec::from_iter(expected);
381
382 for has_columns in [false, true] {
383 for fragment_id in [0, 10] {
384 let mut datagen = lance_datagen::gen();
386 if has_columns {
387 datagen = datagen.col("x", lance_datagen::array::rand::<Int32Type>());
388 }
389 let data = batch_task_stream(
390 datagen
391 .into_reader_stream(RowCount::from(10), BatchCount::from(10))
392 .0,
393 );
394
395 let config = RowIdAndDeletesConfig {
396 params: params.clone(),
397 with_row_id: true,
398 with_row_addr: false,
399 deletion_vector: None,
400 row_id_sequence: None,
401 make_deletions_null: false,
402 total_num_rows: 100,
403 };
404 let stream = super::wrap_with_row_id_and_delete(data, fragment_id, config);
405 let batches = stream.buffered(1).try_collect::<Vec<_>>().await.unwrap();
406
407 let mut offset = 0;
408 let expected = expected.clone();
409 for batch in batches {
410 let actual_row_ids =
411 batch[ROW_ID].as_primitive::<UInt64Type>().values().to_vec();
412 let expected_row_ids = expected[offset..offset + 10]
413 .iter()
414 .map(|row_offset| {
415 RowAddress::new_from_parts(fragment_id, *row_offset).into()
416 })
417 .collect::<Vec<u64>>();
418 assert_eq!(actual_row_ids, expected_row_ids);
419 offset += batch.num_rows();
420 }
421 }
422 }
423 }
424
425 #[tokio::test]
426 async fn test_row_id() {
427 let some_indices = (0..100).rev().collect::<Vec<u32>>();
428 let some_indices_arr = UInt32Array::from(some_indices.clone());
429 check_row_id(ReadBatchParams::RangeFull, 0..100).await;
430 check_row_id(ReadBatchParams::Indices(some_indices_arr), some_indices).await;
431 check_row_id(ReadBatchParams::Range(1000..1100), 1000..1100).await;
432 check_row_id(
433 ReadBatchParams::RangeFrom(std::ops::RangeFrom { start: 1000 }),
434 1000..1100,
435 )
436 .await;
437 check_row_id(
438 ReadBatchParams::RangeTo(std::ops::RangeTo { end: 1000 }),
439 0..100,
440 )
441 .await;
442 }
443
444 #[tokio::test]
445 async fn test_deletes() {
446 let no_deletes: Option<Arc<DeletionVector>> = None;
447 let no_deletes_2 = Some(Arc::new(DeletionVector::NoDeletions));
448 let delete_some_bitmap = Some(Arc::new(DeletionVector::Bitmap(RoaringBitmap::from_iter(
449 0..35,
450 ))));
451 let delete_some_set = Some(Arc::new(DeletionVector::Set((0..35).collect())));
452
453 for deletion_vector in [
454 no_deletes,
455 no_deletes_2,
456 delete_some_bitmap,
457 delete_some_set,
458 ] {
459 for has_columns in [false, true] {
460 for with_row_id in [false, true] {
461 for make_deletions_null in [false, true] {
462 for frag_id in [0, 1] {
463 let has_deletions = if let Some(dv) = &deletion_vector {
464 !matches!(dv.as_ref(), DeletionVector::NoDeletions)
465 } else {
466 false
467 };
468 if !has_columns && !has_deletions && !with_row_id {
469 continue;
472 }
473 if make_deletions_null && !with_row_id {
474 continue;
477 }
478
479 let mut datagen = lance_datagen::gen();
480 if has_columns {
481 datagen =
482 datagen.col("x", lance_datagen::array::rand::<Int32Type>());
483 }
484 let data = batch_task_stream(
486 datagen
487 .into_reader_stream(RowCount::from(10), BatchCount::from(10))
488 .0,
489 );
490
491 let config = RowIdAndDeletesConfig {
492 params: ReadBatchParams::RangeFull,
493 with_row_id,
494 with_row_addr: false,
495 deletion_vector: deletion_vector.clone(),
496 row_id_sequence: None,
497 make_deletions_null,
498 total_num_rows: 100,
499 };
500 let stream = super::wrap_with_row_id_and_delete(data, frag_id, config);
501 let batches = stream
502 .buffered(1)
503 .filter_map(|batch| {
504 std::future::ready(
505 batch
506 .map(|batch| {
507 if batch.num_rows() == 0 {
508 None
509 } else {
510 Some(batch)
511 }
512 })
513 .transpose(),
514 )
515 })
516 .try_collect::<Vec<_>>()
517 .await
518 .unwrap();
519
520 let total_num_rows =
521 batches.iter().map(|b| b.num_rows()).sum::<usize>();
522 let total_num_nulls = if make_deletions_null {
523 batches
524 .iter()
525 .map(|b| b[ROW_ID].null_count())
526 .sum::<usize>()
527 } else {
528 0
529 };
530 let total_actually_deleted = total_num_nulls + (100 - total_num_rows);
531
532 let expected_deletions = match &deletion_vector {
533 None => 0,
534 Some(deletion_vector) => match deletion_vector.as_ref() {
535 DeletionVector::NoDeletions => 0,
536 DeletionVector::Bitmap(b) => b.len() as usize,
537 DeletionVector::Set(s) => s.len(),
538 },
539 };
540 assert_eq!(total_actually_deleted, expected_deletions);
541 if expected_deletions > 0 && with_row_id {
542 if make_deletions_null {
543 assert_eq!(
546 batches[3][ROW_ID].as_primitive::<UInt64Type>().value(0),
547 u64::from(RowAddress::new_from_parts(frag_id, 30))
548 );
549 assert_eq!(batches[3][ROW_ID].null_count(), 5);
550 } else {
551 assert_eq!(
553 batches[0][ROW_ID].as_primitive::<UInt64Type>().value(0),
554 u64::from(RowAddress::new_from_parts(frag_id, 35))
555 );
556 }
557 }
558 if !with_row_id {
559 assert!(batches[0].column_by_name(ROW_ID).is_none());
560 }
561 }
562 }
563 }
564 }
565 }
566 }
567}