|
17 | 17 |
|
18 | 18 | //! Conversions between PyArrow and DataFusion types |
19 | 19 |
|
20 | | -use arrow::array::{Array, ArrayData}; |
| 20 | +use std::sync::Arc; |
| 21 | + |
| 22 | +use arrow::array::{make_array, Array, ArrayData, ListArray}; |
| 23 | +use arrow::buffer::OffsetBuffer; |
| 24 | +use arrow::datatypes::Field; |
21 | 25 | use arrow::pyarrow::{FromPyArrow, ToPyArrow}; |
| 26 | +use datafusion::common::exec_err; |
22 | 27 | use datafusion::scalar::ScalarValue; |
23 | 28 | use pyo3::types::{PyAnyMethods, PyList}; |
24 | 29 | use pyo3::{Bound, FromPyObject, PyAny, PyResult, Python}; |
25 | 30 |
|
26 | 31 | use crate::common::data_type::PyScalarValue; |
27 | 32 | use crate::errors::PyDataFusionError; |
28 | 33 |
|
| 34 | +fn pyobj_extract_scalar_via_capsule( |
| 35 | + value: &Bound<'_, PyAny>, |
| 36 | + as_list_array: bool, |
| 37 | +) -> PyResult<PyScalarValue> { |
| 38 | + let array_data = ArrayData::from_pyarrow_bound(value)?; |
| 39 | + let array = make_array(array_data); |
| 40 | + |
| 41 | + if as_list_array { |
| 42 | + let field = Arc::new(Field::new_list_field( |
| 43 | + array.data_type().clone(), |
| 44 | + array.nulls().is_some(), |
| 45 | + )); |
| 46 | + let offsets = OffsetBuffer::from_lengths(vec![array.len()]); |
| 47 | + let list_array = ListArray::new(field, offsets, array, None); |
| 48 | + Ok(PyScalarValue(ScalarValue::List(Arc::new(list_array)))) |
| 49 | + } else { |
| 50 | + let scalar = ScalarValue::try_from_array(&array, 0).map_err(PyDataFusionError::from)?; |
| 51 | + Ok(PyScalarValue(scalar)) |
| 52 | + } |
| 53 | +} |
| 54 | + |
29 | 55 | impl FromPyArrow for PyScalarValue { |
30 | 56 | fn from_pyarrow_bound(value: &Bound<'_, PyAny>) -> PyResult<Self> { |
31 | 57 | let py = value.py(); |
32 | | - let typ = value.getattr("type")?; |
| 58 | + let pyarrow_mod = py.import("pyarrow"); |
33 | 59 |
|
34 | | - // construct pyarrow array from the python value and pyarrow type |
35 | | - let factory = py.import("pyarrow")?.getattr("array")?; |
36 | | - let args = PyList::new(py, [value])?; |
37 | | - let array = factory.call1((args, typ))?; |
| 60 | + // Is it a PyArrow object? |
| 61 | + if let Ok(pa) = pyarrow_mod.as_ref() { |
| 62 | + let scalar_type = pa.getattr("Scalar")?; |
| 63 | + if value.is_instance(&scalar_type)? { |
| 64 | + let typ = value.getattr("type")?; |
38 | 65 |
|
39 | | - // convert the pyarrow array to rust array using C data interface |
40 | | - let array = arrow::array::make_array(ArrayData::from_pyarrow_bound(&array)?); |
41 | | - let scalar = ScalarValue::try_from_array(&array, 0).map_err(PyDataFusionError::from)?; |
| 66 | + // construct pyarrow array from the python value and pyarrow type |
| 67 | + let factory = py.import("pyarrow")?.getattr("array")?; |
| 68 | + let args = PyList::new(py, [value])?; |
| 69 | + let array = factory.call1((args, typ))?; |
42 | 70 |
|
43 | | - Ok(PyScalarValue(scalar)) |
| 71 | + return pyobj_extract_scalar_via_capsule(&array, false); |
| 72 | + } |
| 73 | + |
| 74 | + let array_type = pa.getattr("Array")?; |
| 75 | + if value.is_instance(&array_type)? { |
| 76 | + return pyobj_extract_scalar_via_capsule(value, true); |
| 77 | + } |
| 78 | + } |
| 79 | + |
| 80 | + // Is it a NanoArrow scalar? |
| 81 | + if let Ok(na) = py.import("nanoarrow") { |
| 82 | + let type_name = value.get_type().repr()?; |
| 83 | + if type_name.contains("nanoarrow")? && type_name.contains("Scalar")? { |
| 84 | + return pyobj_extract_scalar_via_capsule(value, false); |
| 85 | + } |
| 86 | + let array_type = na.getattr("Array")?; |
| 87 | + if value.is_instance(&array_type)? { |
| 88 | + return pyobj_extract_scalar_via_capsule(value, true); |
| 89 | + } |
| 90 | + } |
| 91 | + |
| 92 | + // Is it a arro3 scalar? |
| 93 | + if let Ok(arro3) = py.import("arro3").and_then(|arro3| arro3.getattr("core")) { |
| 94 | + let scalar_type = arro3.getattr("Scalar")?; |
| 95 | + if value.is_instance(&scalar_type)? { |
| 96 | + return pyobj_extract_scalar_via_capsule(value, false); |
| 97 | + } |
| 98 | + let array_type = arro3.getattr("Array")?; |
| 99 | + if value.is_instance(&array_type)? { |
| 100 | + return pyobj_extract_scalar_via_capsule(value, true); |
| 101 | + } |
| 102 | + } |
| 103 | + |
| 104 | + // Does it have a PyCapsule interface but isn't one of our known libraries? |
| 105 | + // If so do our "best guess". Try checking type name, and if that fails |
| 106 | + // return a single value if the length is 1 and return a List value otherwise |
| 107 | + if value.hasattr("__arrow_c_array__")? { |
| 108 | + let type_name = value.get_type().repr()?; |
| 109 | + if type_name.contains("Scalar")? { |
| 110 | + return pyobj_extract_scalar_via_capsule(value, false); |
| 111 | + } |
| 112 | + if type_name.contains("Array")? { |
| 113 | + return pyobj_extract_scalar_via_capsule(value, true); |
| 114 | + } |
| 115 | + |
| 116 | + let array_data = ArrayData::from_pyarrow_bound(value)?; |
| 117 | + let array = make_array(array_data); |
| 118 | + if array.len() == 1 { |
| 119 | + let scalar = |
| 120 | + ScalarValue::try_from_array(&array, 0).map_err(PyDataFusionError::from)?; |
| 121 | + return Ok(PyScalarValue(scalar)); |
| 122 | + } else { |
| 123 | + let field = Arc::new(Field::new_list_field( |
| 124 | + array.data_type().clone(), |
| 125 | + array.nulls().is_some(), |
| 126 | + )); |
| 127 | + let offsets = OffsetBuffer::from_lengths(vec![array.len()]); |
| 128 | + let list_array = ListArray::new(field, offsets, array, None); |
| 129 | + return Ok(PyScalarValue(ScalarValue::List(Arc::new(list_array)))); |
| 130 | + } |
| 131 | + } |
| 132 | + |
| 133 | + // Last attempt - try to create a PyArrow scalar from a plain Python object |
| 134 | + if let Ok(pa) = pyarrow_mod.as_ref() { |
| 135 | + let scalar = pa.call_method1("scalar", (value,))?; |
| 136 | + |
| 137 | + PyScalarValue::from_pyarrow_bound(&scalar) |
| 138 | + } else { |
| 139 | + exec_err!("Unable to import scalar value").map_err(PyDataFusionError::from)? |
| 140 | + } |
44 | 141 | } |
45 | 142 | } |
46 | 143 |
|
|
0 commit comments