1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

use std::{fmt::Debug, sync::Arc};

use arrow_array::{
    new_null_array,
    types::{
        ArrowPrimitiveType, Date32Type, Date64Type, Decimal128Type, Decimal256Type,
        DurationMicrosecondType, DurationMillisecondType, DurationNanosecondType,
        DurationSecondType, Float16Type, Float32Type, Float64Type, Int16Type, Int32Type, Int64Type,
        Int8Type, IntervalDayTimeType, IntervalMonthDayNanoType, IntervalYearMonthType,
        Time32MillisecondType, Time32SecondType, Time64MicrosecondType, Time64NanosecondType,
        TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType,
        TimestampSecondType, UInt16Type, UInt32Type, UInt64Type, UInt8Type,
    },
    ArrayRef, BooleanArray, FixedSizeBinaryArray, FixedSizeListArray, PrimitiveArray,
};
use arrow_buffer::{BooleanBuffer, Buffer, NullBuffer, ScalarBuffer};
use arrow_schema::{DataType, IntervalUnit, TimeUnit};
use bytes::BytesMut;
use futures::{future::BoxFuture, FutureExt};
use lance_arrow::deepcopy::deep_copy_array;
use log::{debug, trace};
use snafu::{location, Location};

use lance_core::{Error, Result};

use crate::{
    decoder::{
        DecodeArrayTask, LogicalPageDecoder, LogicalPageScheduler, NextDecodeTask, PageInfo,
        PhysicalPageDecoder, PhysicalPageScheduler, SchedulerContext,
    },
    encoder::{ArrayEncodingStrategy, EncodeTask, EncodedPage, FieldEncoder},
    encodings::physical::{decoder_from_array_encoding, ColumnBuffers, PageBuffers},
};

/// A page scheduler for primitive fields
///
/// This maps to exactly one physical page and it assumes that the top-level
/// encoding of the page is "basic".  The basic encoding decodes into an
/// optional buffer of validity and a fixed-width buffer of values
/// which is exactly what we need to create a primitive array.
///
/// Note: we consider booleans and fixed-size-lists of primitive types to be
/// primitive types.  This is slightly different than arrow-rs's definition
#[derive(Debug)]
pub struct PrimitivePageScheduler {
    data_type: DataType,
    physical_decoder: Box<dyn PhysicalPageScheduler>,
    num_rows: u32,
}

impl PrimitivePageScheduler {
    pub fn new(data_type: DataType, page: Arc<PageInfo>, buffers: ColumnBuffers) -> Self {
        let page_buffers = PageBuffers {
            column_buffers: buffers,
            positions_and_sizes: &page.buffer_offsets_and_sizes,
        };
        Self {
            data_type,
            physical_decoder: decoder_from_array_encoding(&page.encoding, &page_buffers),
            num_rows: page.num_rows,
        }
    }
}

impl LogicalPageScheduler for PrimitivePageScheduler {
    fn num_rows(&self) -> u32 {
        self.num_rows
    }

    fn schedule_ranges(
        &self,
        ranges: &[std::ops::Range<u32>],
        context: &mut SchedulerContext,
        top_level_row: u64,
    ) -> Result<()> {
        let num_rows = ranges.iter().map(|r| r.end - r.start).sum();
        let physical_decoder =
            self.physical_decoder
                .schedule_ranges(ranges, context.io(), top_level_row);

        let logical_decoder = PrimitiveFieldDecoder {
            data_type: self.data_type.clone(),
            unloaded_physical_decoder: Some(physical_decoder),
            physical_decoder: None,
            rows_drained: 0,
            num_rows,
        };

        context.emit(Box::new(logical_decoder));
        Ok(())
    }

    fn schedule_take(
        &self,
        indices: &[u32],
        context: &mut SchedulerContext,
        top_level_row: u64,
    ) -> Result<()> {
        trace!(
            "Scheduling take of {} indices from physical page",
            indices.len()
        );
        self.schedule_ranges(
            &indices
                .iter()
                .map(|&idx| idx..(idx + 1))
                .collect::<Vec<_>>(),
            context,
            top_level_row,
        )
    }
}

struct PrimitiveFieldDecoder {
    data_type: DataType,
    unloaded_physical_decoder: Option<BoxFuture<'static, Result<Box<dyn PhysicalPageDecoder>>>>,
    physical_decoder: Option<Arc<dyn PhysicalPageDecoder>>,
    num_rows: u32,
    rows_drained: u32,
}

impl Debug for PrimitiveFieldDecoder {
    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("PrimitiveFieldDecoder")
            .field("data_type", &self.data_type)
            .field("num_rows", &self.num_rows)
            .field("rows_drained", &self.rows_drained)
            .finish()
    }
}

struct PrimitiveFieldDecodeTask {
    rows_to_skip: u32,
    rows_to_take: u32,
    physical_decoder: Arc<dyn PhysicalPageDecoder>,
    data_type: DataType,
}

impl DecodeArrayTask for PrimitiveFieldDecodeTask {
    fn decode(self: Box<Self>) -> Result<ArrayRef> {
        // We start by assuming that no buffers are required.  The number of buffers needed is based
        // on the data type.  Most data types need two buffers but each layer of fixed-size-list, for
        // example, adds another validity buffer
        let mut capacities = vec![(0, false); self.physical_decoder.num_buffers() as usize];
        let mut all_null = false;
        self.physical_decoder.update_capacity(
            self.rows_to_skip,
            self.rows_to_take,
            &mut capacities,
            &mut all_null,
        );

        if all_null {
            return Ok(new_null_array(&self.data_type, self.rows_to_take as usize));
        }

        // At this point we know the size needed for each buffer
        let mut bufs = capacities
            .into_iter()
            .map(|(num_bytes, is_needed)| {
                // Only allocate the validity buffer if it is needed, otherwise we
                // create an empty BytesMut (does not require allocation)
                if is_needed {
                    BytesMut::with_capacity(num_bytes as usize)
                } else {
                    BytesMut::default()
                }
            })
            .collect::<Vec<_>>();

        // Go ahead and fill the validity / values buffers
        self.physical_decoder
            .decode_into(self.rows_to_skip, self.rows_to_take, &mut bufs)?;

        // Convert the two buffers into an Arrow array
        Self::primitive_array_from_buffers(&self.data_type, bufs, self.rows_to_take)
    }
}

impl PrimitiveFieldDecodeTask {
    // TODO: Does this capability exist upstream somewhere?  I couldn't find
    // it from a simple scan but it seems the ability to convert two buffers
    // into a primitive array is pretty fundamental.
    fn new_primitive_array<T: ArrowPrimitiveType>(
        buffers: Vec<BytesMut>,
        num_rows: u32,
        data_type: &DataType,
    ) -> ArrayRef {
        let mut buffer_iter = buffers.into_iter();
        let null_buffer = buffer_iter.next().unwrap();
        let null_buffer = if null_buffer.is_empty() {
            None
        } else {
            let null_buffer = null_buffer.freeze().into();
            Some(NullBuffer::new(BooleanBuffer::new(
                Buffer::from_bytes(null_buffer),
                0,
                num_rows as usize,
            )))
        };

        let data_buffer = buffer_iter.next().unwrap().freeze();
        let data_buffer = Buffer::from_bytes(data_buffer.into());
        let data_buffer = ScalarBuffer::<T::Native>::new(data_buffer, 0, num_rows as usize);

        // The with_data_type is needed here to recover the parameters for types like Decimal/Timestamp
        Arc::new(
            PrimitiveArray::<T>::new(data_buffer, null_buffer).with_data_type(data_type.clone()),
        )
    }

    fn bytes_to_validity(bytes: BytesMut, num_rows: u32) -> Option<NullBuffer> {
        if bytes.is_empty() {
            None
        } else {
            let null_buffer = bytes.freeze().into();
            Some(NullBuffer::new(BooleanBuffer::new(
                Buffer::from_bytes(null_buffer),
                0,
                num_rows as usize,
            )))
        }
    }

    fn primitive_array_from_buffers(
        data_type: &DataType,
        buffers: Vec<BytesMut>,
        num_rows: u32,
    ) -> Result<ArrayRef> {
        match data_type {
            DataType::Boolean => {
                let mut buffer_iter = buffers.into_iter();
                let null_buffer = buffer_iter.next().unwrap();
                let null_buffer = Self::bytes_to_validity(null_buffer, num_rows);

                let data_buffer = buffer_iter.next().unwrap().freeze();
                let data_buffer = Buffer::from(data_buffer);
                let data_buffer = BooleanBuffer::new(data_buffer, 0, num_rows as usize);

                Ok(Arc::new(BooleanArray::new(data_buffer, null_buffer)))
            }
            DataType::Date32 => Ok(Self::new_primitive_array::<Date32Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Date64 => Ok(Self::new_primitive_array::<Date64Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Decimal128(_, _) => Ok(Self::new_primitive_array::<Decimal128Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Decimal256(_, _) => Ok(Self::new_primitive_array::<Decimal256Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Duration(units) => Ok(match units {
                TimeUnit::Second => {
                    Self::new_primitive_array::<DurationSecondType>(buffers, num_rows, data_type)
                }
                TimeUnit::Microsecond => Self::new_primitive_array::<DurationMicrosecondType>(
                    buffers, num_rows, data_type,
                ),
                TimeUnit::Millisecond => Self::new_primitive_array::<DurationMillisecondType>(
                    buffers, num_rows, data_type,
                ),
                TimeUnit::Nanosecond => Self::new_primitive_array::<DurationNanosecondType>(
                    buffers, num_rows, data_type,
                ),
            }),
            DataType::Float16 => Ok(Self::new_primitive_array::<Float16Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Float32 => Ok(Self::new_primitive_array::<Float32Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Float64 => Ok(Self::new_primitive_array::<Float64Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Int16 => Ok(Self::new_primitive_array::<Int16Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Int32 => Ok(Self::new_primitive_array::<Int32Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Int64 => Ok(Self::new_primitive_array::<Int64Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Int8 => Ok(Self::new_primitive_array::<Int8Type>(
                buffers, num_rows, data_type,
            )),
            DataType::Interval(unit) => Ok(match unit {
                IntervalUnit::DayTime => {
                    Self::new_primitive_array::<IntervalDayTimeType>(buffers, num_rows, data_type)
                }
                IntervalUnit::MonthDayNano => {
                    Self::new_primitive_array::<IntervalMonthDayNanoType>(
                        buffers, num_rows, data_type,
                    )
                }
                IntervalUnit::YearMonth => {
                    Self::new_primitive_array::<IntervalYearMonthType>(buffers, num_rows, data_type)
                }
            }),
            DataType::Null => Ok(new_null_array(data_type, num_rows as usize)),
            DataType::Time32(unit) => match unit {
                TimeUnit::Millisecond => Ok(Self::new_primitive_array::<Time32MillisecondType>(
                    buffers, num_rows, data_type,
                )),
                TimeUnit::Second => Ok(Self::new_primitive_array::<Time32SecondType>(
                    buffers, num_rows, data_type,
                )),
                _ => Err(Error::io(
                    format!("invalid time unit {:?} for 32-bit time type", unit),
                    location!(),
                )),
            },
            DataType::Time64(unit) => match unit {
                TimeUnit::Microsecond => Ok(Self::new_primitive_array::<Time64MicrosecondType>(
                    buffers, num_rows, data_type,
                )),
                TimeUnit::Nanosecond => Ok(Self::new_primitive_array::<Time64NanosecondType>(
                    buffers, num_rows, data_type,
                )),
                _ => Err(Error::io(
                    format!("invalid time unit {:?} for 64-bit time type", unit),
                    location!(),
                )),
            },
            DataType::Timestamp(unit, _) => Ok(match unit {
                TimeUnit::Microsecond => Self::new_primitive_array::<TimestampMicrosecondType>(
                    buffers, num_rows, data_type,
                ),
                TimeUnit::Millisecond => Self::new_primitive_array::<TimestampMillisecondType>(
                    buffers, num_rows, data_type,
                ),
                TimeUnit::Nanosecond => Self::new_primitive_array::<TimestampNanosecondType>(
                    buffers, num_rows, data_type,
                ),
                TimeUnit::Second => {
                    Self::new_primitive_array::<TimestampSecondType>(buffers, num_rows, data_type)
                }
            }),
            DataType::UInt16 => Ok(Self::new_primitive_array::<UInt16Type>(
                buffers, num_rows, data_type,
            )),
            DataType::UInt32 => Ok(Self::new_primitive_array::<UInt32Type>(
                buffers, num_rows, data_type,
            )),
            DataType::UInt64 => Ok(Self::new_primitive_array::<UInt64Type>(
                buffers, num_rows, data_type,
            )),
            DataType::UInt8 => Ok(Self::new_primitive_array::<UInt8Type>(
                buffers, num_rows, data_type,
            )),
            DataType::FixedSizeBinary(dimension) => {
                let mut buffers_iter = buffers.into_iter();
                let fsb_validity = buffers_iter.next().unwrap();
                let fsb_nulls = Self::bytes_to_validity(fsb_validity, num_rows);

                let fsb_values = buffers_iter.next().unwrap();
                let fsb_values = Buffer::from_bytes(fsb_values.freeze().into());
                Ok(Arc::new(FixedSizeBinaryArray::new(
                    *dimension, fsb_values, fsb_nulls,
                )))
            }
            DataType::FixedSizeList(items, dimension) => {
                let mut buffers_iter = buffers.into_iter();
                let fsl_validity = buffers_iter.next().unwrap();
                let fsl_nulls = Self::bytes_to_validity(fsl_validity, num_rows);

                let remaining_buffers = buffers_iter.collect::<Vec<_>>();
                let items_array = Self::primitive_array_from_buffers(
                    items.data_type(),
                    remaining_buffers,
                    num_rows * (*dimension as u32),
                )?;
                Ok(Arc::new(FixedSizeListArray::new(
                    items.clone(),
                    *dimension,
                    items_array,
                    fsl_nulls,
                )))
            }
            _ => Err(Error::io(
                format!(
                    "The data type {} cannot be decoded from a primitive encoding",
                    data_type
                ),
                location!(),
            )),
        }
    }
}

impl LogicalPageDecoder for PrimitiveFieldDecoder {
    // TODO: In the future, at some point, we may consider partially waiting for primitive pages by
    // breaking up large I/O into smaller I/O as a way to accelerate the "time-to-first-decode"
    fn wait(&mut self, _: u32) -> BoxFuture<Result<()>> {
        async move {
            let physical_decoder = self.unloaded_physical_decoder.take().unwrap().await?;
            self.physical_decoder = Some(Arc::from(physical_decoder));
            Ok(())
        }
        .boxed()
    }

    fn drain(&mut self, num_rows: u32) -> Result<NextDecodeTask> {
        let rows_to_skip = self.rows_drained;
        let rows_to_take = num_rows;

        self.rows_drained += rows_to_take;

        let task = Box::new(PrimitiveFieldDecodeTask {
            rows_to_skip,
            rows_to_take,
            physical_decoder: self.physical_decoder.as_ref().unwrap().clone(),
            data_type: self.data_type.clone(),
        });

        Ok(NextDecodeTask {
            task,
            num_rows: rows_to_take,
            has_more: self.rows_drained != self.num_rows,
        })
    }

    fn unawaited(&self) -> u32 {
        if self.unloaded_physical_decoder.is_some() {
            self.num_rows
        } else {
            0
        }
    }

    fn avail(&self) -> u32 {
        if self.unloaded_physical_decoder.is_some() {
            0
        } else {
            self.num_rows - self.rows_drained
        }
    }

    fn data_type(&self) -> &DataType {
        &self.data_type
    }
}

#[derive(Debug)]
pub struct AccumulationQueue {
    cache_bytes: u64,
    keep_original_array: bool,
    buffered_arrays: Vec<ArrayRef>,
    current_bytes: u64,
    // This is only for logging / debugging purposes
    column_index: u32,
}

impl AccumulationQueue {
    pub fn new(cache_bytes: u64, column_index: u32, keep_original_array: bool) -> Self {
        Self {
            cache_bytes,
            buffered_arrays: Vec::new(),
            current_bytes: 0,
            column_index,
            keep_original_array,
        }
    }

    /// Adds an array to the queue, if there is enough data then the queue is flushed
    /// and returned
    pub fn insert(&mut self, array: ArrayRef) -> Option<Vec<ArrayRef>> {
        self.current_bytes += array.get_array_memory_size() as u64;
        if self.current_bytes > self.cache_bytes {
            debug!(
                "Flushing column {} page of size {} bytes (unencoded)",
                self.column_index, self.current_bytes
            );
            // Push into buffered_arrays without copy since we are about to flush anyways
            self.buffered_arrays.push(array);
            self.current_bytes = 0;
            Some(std::mem::take(&mut self.buffered_arrays))
        } else {
            trace!(
                "Accumulating data for column {}.  Now at {} bytes",
                self.column_index,
                self.current_bytes
            );
            if self.keep_original_array {
                self.buffered_arrays.push(array);
            } else {
                self.buffered_arrays.push(deep_copy_array(array.as_ref()))
            }
            None
        }
    }

    pub fn flush(&mut self) -> Option<Vec<ArrayRef>> {
        if self.buffered_arrays.is_empty() {
            trace!(
                "No final flush since no data at column {}",
                self.column_index
            );
            None
        } else {
            trace!(
                "Final flush of column {} which has {} bytes",
                self.column_index,
                self.current_bytes
            );
            self.current_bytes = 0;
            Some(std::mem::take(&mut self.buffered_arrays))
        }
    }
}

pub struct PrimitiveFieldEncoder {
    accumulation_queue: AccumulationQueue,
    array_encoding_strategy: Arc<dyn ArrayEncodingStrategy>,
    column_index: u32,
}

impl PrimitiveFieldEncoder {
    pub fn try_new(
        cache_bytes: u64,
        keep_original_array: bool,
        array_encoding_strategy: Arc<dyn ArrayEncodingStrategy>,
        column_index: u32,
    ) -> Result<Self> {
        Ok(Self {
            accumulation_queue: AccumulationQueue::new(
                cache_bytes,
                column_index,
                keep_original_array,
            ),
            column_index,
            array_encoding_strategy,
        })
    }

    // Creates an encode task, consuming all buffered data
    fn do_flush(&mut self, arrays: Vec<ArrayRef>) -> Result<EncodeTask> {
        let encoder = self.array_encoding_strategy.create_array_encoder(&arrays)?;
        let column_idx = self.column_index;

        Ok(tokio::task::spawn(async move {
            let num_rows = arrays.iter().map(|arr| arr.len() as u32).sum();
            let mut buffer_index = 0;
            let array = encoder.encode(&arrays, &mut buffer_index)?;
            Ok(EncodedPage {
                array,
                num_rows,
                column_idx,
            })
        })
        .map(|res_res| res_res.unwrap())
        .boxed())
    }
}

impl FieldEncoder for PrimitiveFieldEncoder {
    // Buffers data, if there is enough to write a page then we create an encode task
    fn maybe_encode(&mut self, array: ArrayRef) -> Result<Vec<EncodeTask>> {
        if let Some(arrays) = self.accumulation_queue.insert(array) {
            Ok(vec![self.do_flush(arrays)?])
        } else {
            Ok(vec![])
        }
    }

    // If there is any data left in the buffer then create an encode task from it
    fn flush(&mut self) -> Result<Vec<EncodeTask>> {
        if let Some(arrays) = self.accumulation_queue.flush() {
            Ok(vec![self.do_flush(arrays)?])
        } else {
            Ok(vec![])
        }
    }

    fn num_columns(&self) -> u32 {
        1
    }
}