lance_encoding/encodings/logical/
binary.rs

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
// SPDX-License-Identifier: Apache-2.0
// SPDX-FileCopyrightText: Copyright The Lance Authors

use std::sync::Arc;

use arrow_array::{
    cast::AsArray,
    types::{BinaryType, ByteArrayType, LargeBinaryType, LargeUtf8Type, UInt8Type, Utf8Type},
    Array, ArrayRef, GenericByteArray, GenericListArray,
};

use arrow_schema::DataType;
use futures::{future::BoxFuture, FutureExt};
use lance_core::Result;
use log::trace;

use crate::decoder::{
    DecodeArrayTask, DecoderReady, FieldScheduler, FilterExpression, LogicalPageDecoder,
    MessageType, NextDecodeTask, PriorityRange, ScheduledScanLine, SchedulerContext, SchedulingJob,
};

/// Wraps a varbin scheduler and uses a BinaryPageDecoder to cast
/// the result to the appropriate type
#[derive(Debug)]
pub struct BinarySchedulingJob<'a> {
    scheduler: &'a BinaryFieldScheduler,
    inner: Box<dyn SchedulingJob + 'a>,
}

impl<'a> SchedulingJob for BinarySchedulingJob<'a> {
    fn schedule_next(
        &mut self,
        context: &mut SchedulerContext,
        priority: &dyn PriorityRange,
    ) -> Result<ScheduledScanLine> {
        let inner_scan = self.inner.schedule_next(context, priority)?;
        let wrapped_decoders = inner_scan
            .decoders
            .into_iter()
            .map(|message| {
                let decoder = message.into_legacy();
                MessageType::DecoderReady(DecoderReady {
                    path: decoder.path,
                    decoder: Box::new(BinaryPageDecoder {
                        inner: decoder.decoder,
                        data_type: self.scheduler.data_type.clone(),
                    }),
                })
            })
            .collect::<Vec<_>>();
        Ok(ScheduledScanLine {
            decoders: wrapped_decoders,
            rows_scheduled: inner_scan.rows_scheduled,
        })
    }

    fn num_rows(&self) -> u64 {
        self.inner.num_rows()
    }
}

/// A logical scheduler for utf8/binary pages which assumes the data are encoded as List<u8>
#[derive(Debug)]
pub struct BinaryFieldScheduler {
    varbin_scheduler: Arc<dyn FieldScheduler>,
    data_type: DataType,
}

impl BinaryFieldScheduler {
    // Create a new ListPageScheduler
    pub fn new(varbin_scheduler: Arc<dyn FieldScheduler>, data_type: DataType) -> Self {
        Self {
            varbin_scheduler,
            data_type,
        }
    }
}

impl FieldScheduler for BinaryFieldScheduler {
    fn schedule_ranges<'a>(
        &'a self,
        ranges: &[std::ops::Range<u64>],
        filter: &FilterExpression,
    ) -> Result<Box<dyn SchedulingJob + 'a>> {
        trace!("Scheduling binary for {} ranges", ranges.len());
        let varbin_job = self.varbin_scheduler.schedule_ranges(ranges, filter)?;
        Ok(Box::new(BinarySchedulingJob {
            scheduler: self,
            inner: varbin_job,
        }))
    }

    fn num_rows(&self) -> u64 {
        self.varbin_scheduler.num_rows()
    }

    fn initialize<'a>(
        &'a self,
        _filter: &'a FilterExpression,
        _context: &'a SchedulerContext,
    ) -> BoxFuture<'a, Result<()>> {
        // 2.0 schedulers do not need to initialize
        std::future::ready(Ok(())).boxed()
    }
}

#[derive(Debug)]
pub struct BinaryPageDecoder {
    inner: Box<dyn LogicalPageDecoder>,
    data_type: DataType,
}

impl LogicalPageDecoder for BinaryPageDecoder {
    fn wait_for_loaded(&mut self, num_rows: u64) -> BoxFuture<Result<()>> {
        self.inner.wait_for_loaded(num_rows)
    }

    fn drain(&mut self, num_rows: u64) -> Result<NextDecodeTask> {
        let inner_task = self.inner.drain(num_rows)?;
        Ok(NextDecodeTask {
            has_more: inner_task.has_more,
            num_rows: inner_task.num_rows,
            task: Box::new(BinaryArrayDecoder {
                inner: inner_task.task,
                data_type: self.data_type.clone(),
            }),
        })
    }

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

    fn rows_loaded(&self) -> u64 {
        self.inner.rows_loaded()
    }

    fn num_rows(&self) -> u64 {
        self.inner.num_rows()
    }

    fn rows_drained(&self) -> u64 {
        self.inner.rows_drained()
    }
}

pub struct BinaryArrayDecoder {
    inner: Box<dyn DecodeArrayTask>,
    data_type: DataType,
}

impl BinaryArrayDecoder {
    fn from_list_array<T: ByteArrayType>(array: &GenericListArray<T::Offset>) -> ArrayRef {
        let values = array
            .values()
            .as_primitive::<UInt8Type>()
            .values()
            .inner()
            .clone();
        let offsets = array.offsets().clone();
        Arc::new(GenericByteArray::<T>::new(
            offsets,
            values,
            array.nulls().cloned(),
        ))
    }
}

impl DecodeArrayTask for BinaryArrayDecoder {
    fn decode(self: Box<Self>) -> Result<ArrayRef> {
        let data_type = self.data_type;
        let arr = self.inner.decode()?;
        match data_type {
            DataType::Binary => Ok(Self::from_list_array::<BinaryType>(arr.as_list::<i32>())),
            DataType::LargeBinary => Ok(Self::from_list_array::<LargeBinaryType>(
                arr.as_list::<i64>(),
            )),
            DataType::Utf8 => Ok(Self::from_list_array::<Utf8Type>(arr.as_list::<i32>())),
            DataType::LargeUtf8 => Ok(Self::from_list_array::<LargeUtf8Type>(arr.as_list::<i64>())),
            _ => panic!("Binary decoder does not support this data type"),
        }
    }
}