use std::{fmt::Debug, ops::Range, sync::Arc, vec};
use arrow_array::{make_array, ArrayRef};
use arrow_buffer::bit_util;
use arrow_schema::DataType;
use futures::{future::BoxFuture, FutureExt};
use lance_arrow::deepcopy::deep_copy_array;
use log::{debug, trace};
use lance_core::{datatypes::Field, Result};
use crate::{
decoder::{
DecodeArrayTask, FieldScheduler, FilterExpression, LogicalPageDecoder, NextDecodeTask,
PageInfo, PageScheduler, PrimitivePageDecoder, ScheduledScanLine, SchedulerContext,
SchedulingJob,
},
encoder::{
ArrayEncodingStrategy, EncodeTask, EncodedColumn, EncodedPage, EncodingOptions,
FieldEncoder,
},
encodings::physical::{decoder_from_array_encoding, ColumnBuffers, PageBuffers},
};
#[derive(Debug)]
struct PrimitivePage {
scheduler: Box<dyn PageScheduler>,
num_rows: u64,
}
#[derive(Debug)]
pub struct PrimitiveFieldScheduler {
data_type: DataType,
page_schedulers: Vec<PrimitivePage>,
num_rows: u64,
should_validate: bool,
}
impl PrimitiveFieldScheduler {
pub fn new(
data_type: DataType,
pages: Arc<[PageInfo]>,
buffers: ColumnBuffers,
should_validate: bool,
) -> Self {
let page_schedulers = pages
.iter()
.map(|page| {
let page_buffers = PageBuffers {
column_buffers: buffers,
positions_and_sizes: &page.buffer_offsets_and_sizes,
};
let scheduler =
decoder_from_array_encoding(&page.encoding, &page_buffers, &data_type);
PrimitivePage {
scheduler,
num_rows: page.num_rows,
}
})
.collect::<Vec<_>>();
let num_rows = page_schedulers.iter().map(|p| p.num_rows).sum();
Self {
data_type,
page_schedulers,
num_rows,
should_validate,
}
}
}
#[derive(Debug)]
struct PrimitiveFieldSchedulingJob<'a> {
scheduler: &'a PrimitiveFieldScheduler,
ranges: Vec<Range<u64>>,
page_idx: usize,
range_idx: usize,
range_offset: u64,
global_row_offset: u64,
}
impl<'a> PrimitiveFieldSchedulingJob<'a> {
pub fn new(scheduler: &'a PrimitiveFieldScheduler, ranges: Vec<Range<u64>>) -> Self {
Self {
scheduler,
ranges,
page_idx: 0,
range_idx: 0,
range_offset: 0,
global_row_offset: 0,
}
}
}
impl<'a> SchedulingJob for PrimitiveFieldSchedulingJob<'a> {
fn schedule_next(
&mut self,
context: &mut SchedulerContext,
top_level_row: u64,
) -> Result<ScheduledScanLine> {
debug_assert!(self.range_idx < self.ranges.len());
let mut range = self.ranges[self.range_idx].clone();
range.start += self.range_offset;
let mut cur_page = &self.scheduler.page_schedulers[self.page_idx];
trace!(
"Current range is {:?} and current page has {} rows",
range,
cur_page.num_rows
);
while cur_page.num_rows + self.global_row_offset <= range.start {
self.global_row_offset += cur_page.num_rows;
self.page_idx += 1;
trace!("Skipping entire page of {} rows", cur_page.num_rows);
cur_page = &self.scheduler.page_schedulers[self.page_idx];
}
let mut ranges_in_page = Vec::new();
while cur_page.num_rows + self.global_row_offset > range.start {
range.start = range.start.max(self.global_row_offset);
let start_in_page = range.start - self.global_row_offset;
let end_in_page = start_in_page + (range.end - range.start);
let end_in_page = end_in_page.min(cur_page.num_rows);
let last_in_range = (end_in_page + self.global_row_offset) >= range.end;
ranges_in_page.push(start_in_page..end_in_page);
if last_in_range {
self.range_idx += 1;
if self.range_idx == self.ranges.len() {
break;
}
range = self.ranges[self.range_idx].clone();
} else {
break;
}
}
let num_rows_in_next = ranges_in_page.iter().map(|r| r.end - r.start).sum();
trace!(
"Scheduling {} rows across {} ranges from page with {} rows",
num_rows_in_next,
ranges_in_page.len(),
cur_page.num_rows
);
self.global_row_offset += cur_page.num_rows;
self.page_idx += 1;
let physical_decoder =
cur_page
.scheduler
.schedule_ranges(&ranges_in_page, context.io(), top_level_row);
let logical_decoder = PrimitiveFieldDecoder {
data_type: self.scheduler.data_type.clone(),
unloaded_physical_decoder: Some(physical_decoder),
physical_decoder: None,
rows_drained: 0,
num_rows: num_rows_in_next,
should_validate: self.scheduler.should_validate,
};
let decoder = Box::new(logical_decoder);
let decoder_ready = context.locate_decoder(decoder);
Ok(ScheduledScanLine {
decoders: vec![decoder_ready],
rows_scheduled: num_rows_in_next,
})
}
fn num_rows(&self) -> u64 {
self.ranges.iter().map(|r| r.end - r.start).sum()
}
}
impl FieldScheduler for PrimitiveFieldScheduler {
fn num_rows(&self) -> u64 {
self.num_rows
}
fn schedule_ranges<'a>(
&'a self,
ranges: &[std::ops::Range<u64>],
_filter: &FilterExpression,
) -> Result<Box<dyn SchedulingJob + 'a>> {
Ok(Box::new(PrimitiveFieldSchedulingJob::new(
self,
ranges.to_vec(),
)))
}
}
pub struct PrimitiveFieldDecoder {
data_type: DataType,
unloaded_physical_decoder: Option<BoxFuture<'static, Result<Box<dyn PrimitivePageDecoder>>>>,
physical_decoder: Option<Arc<dyn PrimitivePageDecoder>>,
should_validate: bool,
num_rows: u64,
rows_drained: u64,
}
impl PrimitiveFieldDecoder {
pub fn new_from_data(
physical_decoder: Arc<dyn PrimitivePageDecoder>,
data_type: DataType,
num_rows: u64,
should_validate: bool,
) -> Self {
Self {
data_type,
unloaded_physical_decoder: None,
physical_decoder: Some(physical_decoder),
should_validate,
num_rows,
rows_drained: 0,
}
}
}
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: u64,
rows_to_take: u64,
should_validate: bool,
physical_decoder: Arc<dyn PrimitivePageDecoder>,
data_type: DataType,
}
impl DecodeArrayTask for PrimitiveFieldDecodeTask {
fn decode(self: Box<Self>) -> Result<ArrayRef> {
let block = self
.physical_decoder
.decode(self.rows_to_skip, self.rows_to_take)?;
Ok(make_array(
block.into_arrow(self.data_type, self.should_validate)?,
))
}
}
impl LogicalPageDecoder for PrimitiveFieldDecoder {
fn wait(&mut self, _: u64) -> 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: u64) -> 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,
should_validate: self.should_validate,
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) -> u64 {
if self.unloaded_physical_decoder.is_some() {
self.num_rows
} else {
0
}
}
fn avail(&self) -> u64 {
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,
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,
}
}
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
);
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,
field: Field,
max_page_bytes: u64,
}
impl PrimitiveFieldEncoder {
pub fn try_new(
options: &EncodingOptions,
array_encoding_strategy: Arc<dyn ArrayEncodingStrategy>,
column_index: u32,
field: Field,
) -> Result<Self> {
Ok(Self {
accumulation_queue: AccumulationQueue::new(
options.cache_bytes_per_column,
column_index,
options.keep_original_array,
),
column_index,
max_page_bytes: options.max_page_bytes,
array_encoding_strategy,
field,
})
}
fn create_encode_task(&mut self, arrays: Vec<ArrayRef>) -> Result<EncodeTask> {
let encoder = self
.array_encoding_strategy
.create_array_encoder(&arrays, &self.field)?;
let column_idx = self.column_index;
Ok(tokio::task::spawn(async move {
let num_rows = arrays.iter().map(|arr| arr.len() as u64).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())
}
fn do_flush(&mut self, arrays: Vec<ArrayRef>) -> Result<Vec<EncodeTask>> {
if arrays.len() == 1 {
let array = arrays.into_iter().next().unwrap();
let size_bytes = array.get_buffer_memory_size();
let num_parts = bit_util::ceil(size_bytes, self.max_page_bytes as usize);
if num_parts <= 1 {
Ok(vec![self.create_encode_task(vec![array])?])
} else {
let mut tasks = Vec::with_capacity(num_parts);
let mut offset = 0;
let part_size = bit_util::ceil(array.len(), num_parts);
for _ in 0..num_parts {
let avail = array.len() - part_size;
let chunk_size = avail.min(part_size);
let part = array.slice(offset, chunk_size);
let task = self.create_encode_task(vec![part])?;
tasks.push(task);
offset += chunk_size;
}
Ok(tasks)
}
} else {
Ok(vec![self.create_encode_task(arrays)?])
}
}
}
impl FieldEncoder for PrimitiveFieldEncoder {
fn maybe_encode(&mut self, array: ArrayRef) -> Result<Vec<EncodeTask>> {
if let Some(arrays) = self.accumulation_queue.insert(array) {
Ok(self.do_flush(arrays)?)
} else {
Ok(vec![])
}
}
fn flush(&mut self) -> Result<Vec<EncodeTask>> {
if let Some(arrays) = self.accumulation_queue.flush() {
Ok(self.do_flush(arrays)?)
} else {
Ok(vec![])
}
}
fn num_columns(&self) -> u32 {
1
}
fn finish(&mut self) -> BoxFuture<'_, Result<Vec<crate::encoder::EncodedColumn>>> {
std::future::ready(Ok(vec![EncodedColumn::default()])).boxed()
}
}