pyvortex/
dataset.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
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
use std::sync::Arc;

use arrow::array::RecordBatchReader;
use arrow::datatypes::SchemaRef;
use arrow::pyarrow::{IntoPyArrow, ToPyArrow};
use pyo3::exceptions::PyTypeError;
use pyo3::prelude::*;
use pyo3::types::{PyLong, PyString};
use vortex::arrow::infer_schema;
use vortex::dtype::{DType, Field};
use vortex::error::VortexResult;
use vortex::expr::RowFilter;
use vortex::file::{
    read_initial_bytes, LayoutContext, LayoutDeserializer, Projection, VortexReadArrayStream,
    VortexReadBuilder, VortexRecordBatchReader,
};
use vortex::io::{ObjectStoreReadAt, TokioFile, VortexReadAt};
use vortex::sampling_compressor::ALL_ENCODINGS_CONTEXT;
use vortex::ArrayData;

use crate::expr::PyExpr;
use crate::object_store_urls::vortex_read_at_from_url;
use crate::{PyArray, TOKIO_RUNTIME};

pub async fn layout_stream_from_reader<T: VortexReadAt + Unpin>(
    reader: T,
    projection: Projection,
    row_filter: Option<RowFilter>,
    indices: Option<ArrayData>,
) -> VortexResult<VortexReadArrayStream<T>> {
    let mut builder = VortexReadBuilder::new(
        reader,
        LayoutDeserializer::new(
            ALL_ENCODINGS_CONTEXT.clone(),
            LayoutContext::default().into(),
        ),
    )
    .with_projection(projection);

    if let Some(row_filter) = row_filter {
        builder = builder.with_row_filter(row_filter);
    }

    if let Some(indices) = indices {
        builder = builder.with_indices(indices);
    }

    Ok(builder.build().await?.into_stream())
}

pub async fn read_array_from_reader<T: VortexReadAt + Unpin + 'static>(
    reader: T,
    projection: Projection,
    row_filter: Option<RowFilter>,
    indices: Option<ArrayData>,
) -> VortexResult<ArrayData> {
    layout_stream_from_reader(reader, projection, row_filter, indices)
        .await?
        .read_all()
        .await
}

pub async fn read_dtype_from_reader<T: VortexReadAt + Unpin>(reader: T) -> VortexResult<DType> {
    let initial_read = read_initial_bytes(&reader, reader.size().await?).await?;
    Ok(initial_read.dtype())
}

fn projection_from_python(columns: Option<Vec<Bound<PyAny>>>) -> PyResult<Projection> {
    fn field_from_pyany(field: &Bound<PyAny>) -> PyResult<Field> {
        if field.clone().is_instance_of::<PyString>() {
            Ok(Field::from(field.downcast::<PyString>()?.to_str()?))
        } else if field.is_instance_of::<PyLong>() {
            Ok(Field::Index(field.extract()?))
        } else {
            Err(PyTypeError::new_err(format!(
                "projection: expected list of string, int, and None, but found: {}.",
                field,
            )))
        }
    }

    Ok(match columns {
        None => Projection::All,
        Some(columns) => Projection::Flat(
            columns
                .iter()
                .map(field_from_pyany)
                .collect::<PyResult<Vec<Field>>>()?,
        ),
    })
}

fn row_filter_from_python(row_filter: Option<&Bound<PyExpr>>) -> Option<RowFilter> {
    row_filter.map(|x| RowFilter::new(x.borrow().unwrap().clone()))
}

#[pyclass(name = "TokioFileDataset", module = "io")]
pub struct TokioFileDataset {
    file: TokioFile,
    schema: SchemaRef,
}

impl TokioFileDataset {
    pub async fn try_new(path: String) -> VortexResult<Self> {
        let file = TokioFile::open(path)?;
        let schema = Arc::new(infer_schema(&read_dtype_from_reader(file.clone()).await?)?);

        Ok(Self { file, schema })
    }

    async fn async_to_array(
        &self,
        columns: Option<Vec<Bound<'_, PyAny>>>,
        row_filter: Option<&Bound<'_, PyExpr>>,
        indices: Option<&PyArray>,
    ) -> PyResult<PyArray> {
        let inner = read_array_from_reader(
            self.file.clone(),
            projection_from_python(columns)?,
            row_filter_from_python(row_filter),
            indices.map(PyArray::unwrap).cloned(),
        )
        .await?;
        Ok(PyArray::new(inner))
    }

    async fn async_to_record_batch_reader(
        self_: PyRef<'_, Self>,
        columns: Option<Vec<Bound<'_, PyAny>>>,
        row_filter: Option<&Bound<'_, PyExpr>>,
        indices: Option<&PyArray>,
    ) -> PyResult<PyObject> {
        let layout_reader = layout_stream_from_reader(
            self_.file.clone(),
            projection_from_python(columns)?,
            row_filter_from_python(row_filter),
            indices.map(PyArray::unwrap).cloned(),
        )
        .await?;

        let record_batch_reader: Box<dyn RecordBatchReader + Send> = Box::new(
            VortexRecordBatchReader::try_new(layout_reader, &*TOKIO_RUNTIME)?,
        );
        record_batch_reader.into_pyarrow(self_.py())
    }
}

#[pymethods]
impl TokioFileDataset {
    fn schema(self_: PyRef<Self>) -> PyResult<PyObject> {
        self_.schema.clone().to_pyarrow(self_.py())
    }

    #[pyo3(signature = (*, columns = None, row_filter = None, indices = None))]
    pub fn to_array(
        &self,
        columns: Option<Vec<Bound<'_, PyAny>>>,
        row_filter: Option<&Bound<'_, PyExpr>>,
        indices: Option<&PyArray>,
    ) -> PyResult<PyArray> {
        TOKIO_RUNTIME.block_on(self.async_to_array(columns, row_filter, indices))
    }

    #[pyo3(signature = (*, columns = None, row_filter = None, indices = None))]
    pub fn to_record_batch_reader(
        self_: PyRef<Self>,
        columns: Option<Vec<Bound<'_, PyAny>>>,
        row_filter: Option<&Bound<'_, PyExpr>>,
        indices: Option<&PyArray>,
    ) -> PyResult<PyObject> {
        TOKIO_RUNTIME.block_on(Self::async_to_record_batch_reader(
            self_, columns, row_filter, indices,
        ))
    }
}

#[pyclass(name = "ObjectStoreUrlDataset", module = "io")]
pub struct ObjectStoreUrlDataset {
    url: String,
    schema: SchemaRef,
}

impl ObjectStoreUrlDataset {
    async fn reader(&self) -> VortexResult<ObjectStoreReadAt> {
        vortex_read_at_from_url(&self.url).await
    }

    pub async fn try_new(url: String) -> VortexResult<Self> {
        let reader = vortex_read_at_from_url(&url).await?;
        let schema = Arc::new(infer_schema(&read_dtype_from_reader(reader).await?)?);

        Ok(Self { url, schema })
    }

    async fn async_to_array(
        &self,
        columns: Option<Vec<Bound<'_, PyAny>>>,
        row_filter: Option<&Bound<'_, PyExpr>>,
        indices: Option<&PyArray>,
    ) -> PyResult<PyArray> {
        let inner = read_array_from_reader(
            self.reader().await?,
            projection_from_python(columns)?,
            row_filter_from_python(row_filter),
            indices.map(PyArray::unwrap).cloned(),
        )
        .await?;
        Ok(PyArray::new(inner))
    }

    async fn async_to_record_batch_reader(
        self_: PyRef<'_, Self>,
        columns: Option<Vec<Bound<'_, PyAny>>>,
        row_filter: Option<&Bound<'_, PyExpr>>,
        indices: Option<&PyArray>,
    ) -> PyResult<PyObject> {
        let layout_reader = layout_stream_from_reader(
            self_.reader().await?,
            projection_from_python(columns)?,
            row_filter_from_python(row_filter),
            indices.map(PyArray::unwrap).cloned(),
        )
        .await?;

        let record_batch_reader: Box<dyn RecordBatchReader + Send> = Box::new(
            VortexRecordBatchReader::try_new(layout_reader, &*TOKIO_RUNTIME)?,
        );
        record_batch_reader.into_pyarrow(self_.py())
    }
}

#[pymethods]
impl ObjectStoreUrlDataset {
    fn schema(self_: PyRef<Self>) -> PyResult<PyObject> {
        self_.schema.clone().to_pyarrow(self_.py())
    }

    #[pyo3(signature = (*, columns = None, row_filter = None, indices = None))]
    pub fn to_array(
        &self,
        columns: Option<Vec<Bound<'_, PyAny>>>,
        row_filter: Option<&Bound<'_, PyExpr>>,
        indices: Option<&PyArray>,
    ) -> PyResult<PyArray> {
        TOKIO_RUNTIME.block_on(self.async_to_array(columns, row_filter, indices))
    }

    #[pyo3(signature = (*, columns = None, row_filter = None, indices = None))]
    pub fn to_record_batch_reader(
        self_: PyRef<Self>,
        columns: Option<Vec<Bound<'_, PyAny>>>,
        row_filter: Option<&Bound<'_, PyExpr>>,
        indices: Option<&PyArray>,
    ) -> PyResult<PyObject> {
        TOKIO_RUNTIME.block_on(Self::async_to_record_batch_reader(
            self_, columns, row_filter, indices,
        ))
    }
}

#[pyfunction]
pub fn dataset_from_url(url: Bound<PyString>) -> PyResult<ObjectStoreUrlDataset> {
    Ok(TOKIO_RUNTIME.block_on(ObjectStoreUrlDataset::try_new(url.extract()?))?)
}

#[pyfunction]
pub fn dataset_from_path(path: Bound<PyString>) -> PyResult<TokioFileDataset> {
    Ok(TOKIO_RUNTIME.block_on(TokioFileDataset::try_new(path.extract()?))?)
}