Path: blob/main/crates/polars-python/src/conversion/any_value.rs
7889 views
use std::borrow::{Borrow, Cow};1use std::sync::{Arc, Mutex};23use chrono::{4DateTime, Datelike, FixedOffset, NaiveDate, NaiveDateTime, NaiveTime, TimeDelta, Timelike,5};6use chrono_tz::Tz;7use hashbrown::HashMap;8use num_traits::ToPrimitive;9#[cfg(feature = "object")]10use polars::chunked_array::object::PolarsObjectSafe;11#[cfg(feature = "object")]12use polars::datatypes::OwnedObject;13use polars::datatypes::{DataType, Field, TimeUnit};14use polars::prelude::{AnyValue, PlSmallStr, Series, TimeZone};15use polars_compute::decimal::{DEC128_MAX_PREC, DecimalFmtBuffer, dec128_fits};16use polars_core::utils::any_values_to_supertype_and_n_dtypes;17use polars_core::utils::arrow::temporal_conversions::date32_to_date;18use polars_utils::aliases::PlFixedStateQuality;19use pyo3::exceptions::{PyOverflowError, PyTypeError, PyValueError};20use pyo3::prelude::*;21use pyo3::sync::PyOnceLock;22use pyo3::types::{23PyBool, PyBytes, PyDate, PyDateTime, PyDelta, PyDict, PyFloat, PyInt, PyList, PyMapping,24PyRange, PySequence, PyString, PyTime, PyTuple, PyType, PyTzInfo,25};26use pyo3::{IntoPyObjectExt, PyTypeCheck, intern};2728use super::datetime::{29datetime_to_py_object, elapsed_offset_to_timedelta, nanos_since_midnight_to_naivetime,30};31use super::{ObjectValue, Wrap, struct_dict};32use crate::error::PyPolarsErr;33use crate::py_modules::{pl_series, pl_utils};34use crate::series::PySeries;3536impl<'py> IntoPyObject<'py> for Wrap<AnyValue<'_>> {37type Target = PyAny;38type Output = Bound<'py, Self::Target>;39type Error = PyErr;4041fn into_pyobject(self, py: Python<'py>) -> Result<Self::Output, Self::Error> {42any_value_into_py_object(self.0, py)43}44}4546impl<'py> IntoPyObject<'py> for &Wrap<AnyValue<'_>> {47type Target = PyAny;48type Output = Bound<'py, Self::Target>;49type Error = PyErr;5051fn into_pyobject(self, py: Python<'py>) -> Result<Self::Output, Self::Error> {52self.clone().into_pyobject(py)53}54}5556impl<'py> FromPyObject<'py> for Wrap<AnyValue<'static>> {57fn extract_bound(ob: &Bound<'py, PyAny>) -> PyResult<Self> {58py_object_to_any_value(ob, true, true).map(Wrap)59}60}6162pub(crate) fn any_value_into_py_object<'py>(63av: AnyValue<'_>,64py: Python<'py>,65) -> PyResult<Bound<'py, PyAny>> {66let utils = pl_utils(py).bind(py);67match av {68AnyValue::UInt8(v) => v.into_bound_py_any(py),69AnyValue::UInt16(v) => v.into_bound_py_any(py),70AnyValue::UInt32(v) => v.into_bound_py_any(py),71AnyValue::UInt64(v) => v.into_bound_py_any(py),72AnyValue::UInt128(v) => v.into_bound_py_any(py),73AnyValue::Int8(v) => v.into_bound_py_any(py),74AnyValue::Int16(v) => v.into_bound_py_any(py),75AnyValue::Int32(v) => v.into_bound_py_any(py),76AnyValue::Int64(v) => v.into_bound_py_any(py),77AnyValue::Int128(v) => v.into_bound_py_any(py),78AnyValue::Float16(v) => v.to_f32().into_bound_py_any(py),79AnyValue::Float32(v) => v.into_bound_py_any(py),80AnyValue::Float64(v) => v.into_bound_py_any(py),81AnyValue::Null => py.None().into_bound_py_any(py),82AnyValue::Boolean(v) => v.into_bound_py_any(py),83AnyValue::String(v) => v.into_bound_py_any(py),84AnyValue::StringOwned(v) => v.into_bound_py_any(py),85AnyValue::Categorical(cat, map) | AnyValue::Enum(cat, map) => unsafe {86map.cat_to_str_unchecked(cat).into_bound_py_any(py)87},88AnyValue::CategoricalOwned(cat, map) | AnyValue::EnumOwned(cat, map) => unsafe {89map.cat_to_str_unchecked(cat).into_bound_py_any(py)90},91AnyValue::Date(v) => {92let date = date32_to_date(v);93date.into_bound_py_any(py)94},95AnyValue::Datetime(v, time_unit, time_zone) => {96datetime_to_py_object(py, v, time_unit, time_zone)97},98AnyValue::DatetimeOwned(v, time_unit, time_zone) => {99datetime_to_py_object(py, v, time_unit, time_zone.as_ref().map(AsRef::as_ref))100},101AnyValue::Duration(v, time_unit) => {102let time_delta = elapsed_offset_to_timedelta(v, time_unit);103time_delta.into_bound_py_any(py)104},105AnyValue::Time(v) => nanos_since_midnight_to_naivetime(v).into_bound_py_any(py),106AnyValue::Array(v, _) | AnyValue::List(v) => PySeries::new(v).to_list(py),107ref av @ AnyValue::Struct(_, _, flds) => {108Ok(struct_dict(py, av._iter_struct_av(), flds)?.into_any())109},110AnyValue::StructOwned(payload) => {111Ok(struct_dict(py, payload.0.into_iter(), &payload.1)?.into_any())112},113#[cfg(feature = "object")]114AnyValue::Object(v) => {115let object = v.as_any().downcast_ref::<ObjectValue>().unwrap();116Ok(object.inner.clone_ref(py).into_bound(py))117},118#[cfg(feature = "object")]119AnyValue::ObjectOwned(v) => {120let object = v.0.as_any().downcast_ref::<ObjectValue>().unwrap();121Ok(object.inner.clone_ref(py).into_bound(py))122},123AnyValue::Binary(v) => PyBytes::new(py, v).into_bound_py_any(py),124AnyValue::BinaryOwned(v) => PyBytes::new(py, &v).into_bound_py_any(py),125AnyValue::Decimal(v, prec, scale) => {126let convert = utils.getattr(intern!(py, "to_py_decimal"))?;127let mut buf = DecimalFmtBuffer::new();128let s = buf.format_dec128(v, scale, false, false);129convert.call1((prec, s))130},131}132}133134/// Holds a Python type object and implements hashing / equality based on the pointer address of the135/// type object. This is used as a hashtable key instead of only the `usize` pointer value, as we136/// need to hold a ref to the Python type object to keep it alive.137#[derive(Debug)]138pub struct TypeObjectKey {139#[allow(unused)]140type_object: Py<PyType>,141/// We need to store this in a field for `Borrow<usize>`142address: usize,143}144145impl TypeObjectKey {146fn new(type_object: Py<PyType>) -> Self {147let address = type_object.as_ptr() as usize;148Self {149type_object,150address,151}152}153}154155impl PartialEq for TypeObjectKey {156fn eq(&self, other: &Self) -> bool {157self.address == other.address158}159}160161impl Eq for TypeObjectKey {}162163impl std::borrow::Borrow<usize> for TypeObjectKey {164fn borrow(&self) -> &usize {165&self.address166}167}168169impl std::hash::Hash for TypeObjectKey {170fn hash<H: std::hash::Hasher>(&self, state: &mut H) {171let v: &usize = self.borrow();172v.hash(state)173}174}175176type InitFn = fn(&Bound<'_, PyAny>, bool) -> PyResult<AnyValue<'static>>;177pub(crate) static LUT: Mutex<HashMap<TypeObjectKey, InitFn, PlFixedStateQuality>> =178Mutex::new(HashMap::with_hasher(PlFixedStateQuality::with_seed(0)));179180/// Convert a Python object to an [`AnyValue`].181pub(crate) fn py_object_to_any_value(182ob: &Bound<'_, PyAny>,183strict: bool,184allow_object: bool,185) -> PyResult<AnyValue<'static>> {186// Conversion functions.187fn get_null(_ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {188Ok(AnyValue::Null)189}190191fn get_bool(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {192let b = ob.extract::<bool>()?;193Ok(AnyValue::Boolean(b))194}195196fn get_int(ob: &Bound<'_, PyAny>, strict: bool) -> PyResult<AnyValue<'static>> {197if let Ok(v) = ob.extract::<i64>() {198Ok(AnyValue::Int64(v))199} else if let Ok(v) = ob.extract::<i128>() {200Ok(AnyValue::Int128(v))201} else if let Ok(v) = ob.extract::<u64>() {202Ok(AnyValue::UInt64(v))203} else if let Ok(v) = ob.extract::<u128>() {204Ok(AnyValue::UInt128(v))205} else if !strict {206let f = ob.extract::<f64>()?;207Ok(AnyValue::Float64(f))208} else {209Err(PyOverflowError::new_err(format!(210"int value too large for Polars integer types: {ob}"211)))212}213}214215fn get_float(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {216Ok(AnyValue::Float64(ob.extract::<f64>()?))217}218219fn get_str(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {220// Ideally we'd be returning an AnyValue::String(&str) instead, as was221// the case in previous versions of this function. However, if compiling222// with abi3 for versions older than Python 3.10, the APIs that purport223// to return &str actually just encode to UTF-8 as a newly allocated224// PyBytes object, and then return reference to that. So what we're225// doing here isn't any different fundamentally, and the APIs to for226// converting to &str are deprecated in PyO3 0.21.227//228// Once Python 3.10 is the minimum supported version, converting to &str229// will be cheaper, and we should do that. Python 3.9 security updates230// end-of-life is Oct 31, 2025.231Ok(AnyValue::StringOwned(ob.extract::<String>()?.into()))232}233234fn get_bytes(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {235let value = ob.extract::<Vec<u8>>()?;236Ok(AnyValue::BinaryOwned(value))237}238239fn get_date(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {240const UNIX_EPOCH: NaiveDate = DateTime::UNIX_EPOCH.naive_utc().date();241let date = ob.extract::<NaiveDate>()?;242let elapsed = date.signed_duration_since(UNIX_EPOCH);243Ok(AnyValue::Date(elapsed.num_days() as i32))244}245246fn get_datetime(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {247let py = ob.py();248let tzinfo = ob.getattr(intern!(py, "tzinfo"))?;249250if tzinfo.is_none() {251let datetime = ob.extract::<NaiveDateTime>()?;252let delta = datetime - DateTime::UNIX_EPOCH.naive_utc();253let timestamp = delta.num_microseconds().unwrap();254return Ok(AnyValue::Datetime(timestamp, TimeUnit::Microseconds, None));255}256257// Try converting `pytz` timezone to `zoneinfo` timezone258let (ob, tzinfo) = if let Some(tz) = tzinfo259.getattr(intern!(py, "zone"))260.ok()261.and_then(|tz| (!tz.is_none()).then_some(tz))262{263let tzinfo = PyTzInfo::timezone(py, tz.downcast_into::<PyString>()?)?;264(265&ob.call_method(intern!(py, "astimezone"), (&tzinfo,), None)?,266tzinfo,267)268} else {269(ob, tzinfo.downcast_into()?)270};271272let (timestamp, tz) = if tzinfo.hasattr(intern!(py, "key"))? {273let datetime = ob.extract::<DateTime<Tz>>()?;274let tz = unsafe { TimeZone::from_static(datetime.timezone().name()) };275if datetime.year() >= 2100 {276// chrono-tz does not support dates after 2100277// https://github.com/chronotope/chrono-tz/issues/135278(279pl_utils(py)280.bind(py)281.getattr(intern!(py, "datetime_to_int"))?282.call1((ob, intern!(py, "us")))?283.extract::<i64>()?,284tz,285)286} else {287let delta = datetime.to_utc() - DateTime::UNIX_EPOCH;288(delta.num_microseconds().unwrap(), tz)289}290} else {291let datetime = ob.extract::<DateTime<FixedOffset>>()?;292let delta = datetime.to_utc() - DateTime::UNIX_EPOCH;293(delta.num_microseconds().unwrap(), TimeZone::UTC)294};295296Ok(AnyValue::DatetimeOwned(297timestamp,298TimeUnit::Microseconds,299Some(Arc::new(tz)),300))301}302303fn get_timedelta(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {304let timedelta = ob.extract::<TimeDelta>()?;305if let Some(micros) = timedelta.num_microseconds() {306Ok(AnyValue::Duration(micros, TimeUnit::Microseconds))307} else {308Ok(AnyValue::Duration(309timedelta.num_milliseconds(),310TimeUnit::Milliseconds,311))312}313}314315fn get_time(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {316let time = ob.extract::<NaiveTime>()?;317318Ok(AnyValue::Time(319(time.num_seconds_from_midnight() as i64) * 1_000_000_000 + time.nanosecond() as i64,320))321}322323fn get_decimal(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {324fn abs_decimal_from_digits(325digits: impl IntoIterator<Item = u8>,326exp: i32,327) -> Option<(i128, usize)> {328let mut v = 0_i128;329for d in digits {330v = v.checked_mul(10)?.checked_add(d as i128)?;331}332let scale = if exp > 0 {333v = 10_i128.checked_pow(exp as u32)?.checked_mul(v)?;3340335} else {336(-exp) as usize337};338dec128_fits(v, DEC128_MAX_PREC).then_some((v, scale))339}340341// Note: Using Vec<u8> is not the most efficient thing here (input is a tuple)342let (sign, digits, exp): (i8, Vec<u8>, i32) = ob343.call_method0(intern!(ob.py(), "as_tuple"))344.unwrap()345.extract()346.unwrap();347let (mut v, scale) = abs_decimal_from_digits(digits, exp).ok_or_else(|| {348PyErr::from(PyPolarsErr::Other(349"Decimal is too large to fit in Decimal128".into(),350))351})?;352if sign > 0 {353v = -v; // Won't overflow since -i128::MAX > i128::MIN354}355Ok(AnyValue::Decimal(v, DEC128_MAX_PREC, scale))356}357358fn get_list(ob: &Bound<'_, PyAny>, strict: bool) -> PyResult<AnyValue<'static>> {359fn get_list_with_constructor(360ob: &Bound<'_, PyAny>,361strict: bool,362) -> PyResult<AnyValue<'static>> {363// Use the dedicated constructor.364// This constructor is able to go via dedicated type constructors365// so it can be much faster.366let py = ob.py();367let kwargs = PyDict::new(py);368kwargs.set_item("strict", strict)?;369let s = pl_series(py).call(py, (ob,), Some(&kwargs))?;370get_list_from_series(s.bind(py), strict)371}372373if ob.is_empty()? {374Ok(AnyValue::List(Series::new_empty(375PlSmallStr::EMPTY,376&DataType::Null,377)))378} else if ob.is_instance_of::<PyList>() | ob.is_instance_of::<PyTuple>() {379let list = ob.downcast::<PySequence>()?;380381// Try to find first non-null.382let length = list.len()?;383let mut iter = list.try_iter()?;384let mut avs = Vec::new();385for item in &mut iter {386let av = py_object_to_any_value(&item?, strict, true)?;387let is_null = av.is_null();388avs.push(av);389if is_null {390break;391}392}393394// Try to use a faster converter.395if let Some(av) = avs.last()396&& !av.is_null()397&& av.dtype().is_primitive()398{399// Always use strict, we will filter the error if we're not400// strict and try again using a slower converter with supertype.401match get_list_with_constructor(ob, true) {402Ok(ret) => return Ok(ret),403Err(e) => {404if strict {405return Err(e);406}407},408}409}410411// Push the rest of the anyvalues and use slower converter.412avs.reserve(length);413for item in &mut iter {414avs.push(py_object_to_any_value(&item?, strict, true)?);415}416417let (dtype, _n_dtypes) = any_values_to_supertype_and_n_dtypes(&avs)418.map_err(|e| PyTypeError::new_err(e.to_string()))?;419let s = Series::from_any_values_and_dtype(PlSmallStr::EMPTY, &avs, &dtype, strict)420.map_err(|e| {421PyTypeError::new_err(format!(422"{e}\n\nHint: Try setting `strict=False` to allow passing data with mixed types."423))424})?;425Ok(AnyValue::List(s))426} else {427// range will take this branch428get_list_with_constructor(ob, strict)429}430}431432fn get_list_from_series(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {433let s = super::get_series(ob)?;434Ok(AnyValue::List(s))435}436437fn get_mapping(ob: &Bound<'_, PyAny>, strict: bool) -> PyResult<AnyValue<'static>> {438let mapping = ob.downcast::<PyMapping>()?;439let len = mapping.len()?;440let mut keys = Vec::with_capacity(len);441let mut vals = Vec::with_capacity(len);442443for item in mapping.items()?.try_iter()? {444let item = item?.downcast_into::<PyTuple>()?;445let (key_py, val_py) = (item.get_item(0)?, item.get_item(1)?);446447let key: Cow<str> = key_py.extract()?;448let val = py_object_to_any_value(&val_py, strict, true)?;449450keys.push(Field::new(key.as_ref().into(), val.dtype()));451vals.push(val);452}453Ok(AnyValue::StructOwned(Box::new((vals, keys))))454}455456fn get_struct(ob: &Bound<'_, PyAny>, strict: bool) -> PyResult<AnyValue<'static>> {457let dict = ob.downcast::<PyDict>().unwrap();458let len = dict.len();459let mut keys = Vec::with_capacity(len);460let mut vals = Vec::with_capacity(len);461for (k, v) in dict.into_iter() {462let key = k.extract::<Cow<str>>()?;463let val = py_object_to_any_value(&v, strict, true)?;464let dtype = val.dtype();465keys.push(Field::new(key.as_ref().into(), dtype));466vals.push(val)467}468Ok(AnyValue::StructOwned(Box::new((vals, keys))))469}470471fn get_namedtuple(ob: &Bound<'_, PyAny>, strict: bool) -> PyResult<AnyValue<'static>> {472let tuple = ob.downcast::<PyTuple>().unwrap();473let len = tuple.len();474let fields = ob475.getattr(intern!(ob.py(), "_fields"))?476.downcast_into::<PyTuple>()?;477let mut keys = Vec::with_capacity(len);478let mut vals = Vec::with_capacity(len);479for (k, v) in fields.into_iter().zip(tuple.into_iter()) {480let key = k.extract::<Cow<str>>()?;481let val = py_object_to_any_value(&v, strict, true)?;482let dtype = val.dtype();483keys.push(Field::new(key.as_ref().into(), dtype));484vals.push(val)485}486Ok(AnyValue::StructOwned(Box::new((vals, keys))))487}488489fn get_object(ob: &Bound<'_, PyAny>, _strict: bool) -> PyResult<AnyValue<'static>> {490#[cfg(feature = "object")]491{492// This is slow, but hey don't use objects.493let v = &ObjectValue {494inner: ob.clone().unbind(),495};496Ok(AnyValue::ObjectOwned(OwnedObject(v.to_boxed())))497}498#[cfg(not(feature = "object"))]499panic!("activate object")500}501502/// Determine which conversion function to use for the given object.503///504/// Note: This function is only ran if the object's type is not already in the505/// lookup table.506fn get_conversion_function(ob: &Bound<'_, PyAny>, allow_object: bool) -> PyResult<InitFn> {507let py = ob.py();508if ob.is_none() {509Ok(get_null)510}511// bool must be checked before int because Python bool is an instance of int.512else if ob.is_instance_of::<PyBool>() {513Ok(get_bool)514} else if ob.is_instance_of::<PyInt>() {515Ok(get_int)516} else if ob.is_instance_of::<PyFloat>() {517Ok(get_float)518} else if ob.is_instance_of::<PyString>() {519Ok(get_str)520} else if ob.is_instance_of::<PyBytes>() {521Ok(get_bytes)522} else if ob.is_instance_of::<PyTuple>() {523// NamedTuple-like object?524if ob.hasattr(intern!(py, "_fields"))? {525Ok(get_namedtuple)526} else {527Ok(get_list)528}529} else if ob.is_instance_of::<PyList>() {530Ok(get_list)531} else if ob.is_instance_of::<PyDict>() {532Ok(get_struct)533} else if PyMapping::type_check(ob) {534Ok(get_mapping)535}536// note: datetime must be checked *before* date537// (as python datetime is an instance of date)538else if PyDateTime::type_check(ob) {539Ok(get_datetime as InitFn)540} else if PyDate::type_check(ob) {541Ok(get_date as InitFn)542} else if PyTime::type_check(ob) {543Ok(get_time as InitFn)544} else if PyDelta::type_check(ob) {545Ok(get_timedelta as InitFn)546} else if ob.is_instance_of::<PyRange>() {547Ok(get_list as InitFn)548} else if ob.is_instance(pl_series(py).bind(py))? {549Ok(get_list_from_series as InitFn)550} else {551static NDARRAY_TYPE: PyOnceLock<Py<PyType>> = PyOnceLock::new();552if let Ok(ndarray_type) = NDARRAY_TYPE.import(py, "numpy", "ndarray") {553if ob.is_instance(ndarray_type)? {554// will convert via Series -> mmap_numpy_array555return Ok(get_list as InitFn);556}557}558static DECIMAL_TYPE: PyOnceLock<Py<PyType>> = PyOnceLock::new();559if ob.is_instance(DECIMAL_TYPE.import(py, "decimal", "Decimal")?)? {560return Ok(get_decimal as InitFn);561}562563// support NumPy scalars564if ob.extract::<i64>().is_ok() || ob.extract::<u64>().is_ok() {565return Ok(get_int as InitFn);566} else if ob.extract::<f64>().is_ok() {567return Ok(get_float as InitFn);568}569570if allow_object {571Ok(get_object as InitFn)572} else {573Err(PyValueError::new_err(format!("Cannot convert {ob}")))574}575}576}577578let py_type = ob.get_type();579let py_type_address = py_type.as_ptr() as usize;580581let conversion_func = {582if let Some(cached_func) = LUT.lock().unwrap().get(&py_type_address) {583*cached_func584} else {585let k = TypeObjectKey::new(py_type.clone().unbind());586assert_eq!(k.address, py_type_address);587588let func = get_conversion_function(ob, allow_object)?;589LUT.lock().unwrap().insert(k, func);590func591}592};593594conversion_func(ob, strict)595}596597598