datafusion_python/
utils.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18use crate::errors::py_datafusion_err;
19use crate::{
20    common::data_type::PyScalarValue,
21    errors::{PyDataFusionError, PyDataFusionResult},
22    TokioRuntime,
23};
24use datafusion::{
25    common::ScalarValue, datasource::TableProvider, execution::context::SessionContext,
26    logical_expr::Volatility,
27};
28use datafusion_ffi::table_provider::{FFI_TableProvider, ForeignTableProvider};
29use pyo3::prelude::*;
30use pyo3::{exceptions::PyValueError, types::PyCapsule};
31use std::{
32    future::Future,
33    sync::{Arc, OnceLock},
34    time::Duration,
35};
36use tokio::{runtime::Runtime, time::sleep};
37
38/// Utility to get the Tokio Runtime from Python
39#[inline]
40pub(crate) fn get_tokio_runtime() -> &'static TokioRuntime {
41    // NOTE: Other pyo3 python libraries have had issues with using tokio
42    // behind a forking app-server like `gunicorn`
43    // If we run into that problem, in the future we can look to `delta-rs`
44    // which adds a check in that disallows calls from a forked process
45    // https://github.com/delta-io/delta-rs/blob/87010461cfe01563d91a4b9cd6fa468e2ad5f283/python/src/utils.rs#L10-L31
46    static RUNTIME: OnceLock<TokioRuntime> = OnceLock::new();
47    RUNTIME.get_or_init(|| TokioRuntime(tokio::runtime::Runtime::new().unwrap()))
48}
49
50#[inline]
51pub(crate) fn is_ipython_env(py: Python) -> &'static bool {
52    static IS_IPYTHON_ENV: OnceLock<bool> = OnceLock::new();
53    IS_IPYTHON_ENV.get_or_init(|| {
54        py.import("IPython")
55            .and_then(|ipython| ipython.call_method0("get_ipython"))
56            .map(|ipython| !ipython.is_none())
57            .unwrap_or(false)
58    })
59}
60
61/// Utility to get the Global Datafussion CTX
62#[inline]
63pub(crate) fn get_global_ctx() -> &'static SessionContext {
64    static CTX: OnceLock<SessionContext> = OnceLock::new();
65    CTX.get_or_init(SessionContext::new)
66}
67
68/// Utility to collect rust futures with GIL released and respond to
69/// Python interrupts such as ``KeyboardInterrupt``. If a signal is
70/// received while the future is running, the future is aborted and the
71/// corresponding Python exception is raised.
72pub fn wait_for_future<F>(py: Python, fut: F) -> PyResult<F::Output>
73where
74    F: Future + Send,
75    F::Output: Send,
76{
77    let runtime: &Runtime = &get_tokio_runtime().0;
78    const INTERVAL_CHECK_SIGNALS: Duration = Duration::from_millis(1_000);
79
80    py.allow_threads(|| {
81        runtime.block_on(async {
82            tokio::pin!(fut);
83            loop {
84                tokio::select! {
85                    res = &mut fut => break Ok(res),
86                    _ = sleep(INTERVAL_CHECK_SIGNALS) => {
87                        Python::with_gil(|py| py.check_signals())?;
88                    }
89                }
90            }
91        })
92    })
93}
94
95pub(crate) fn parse_volatility(value: &str) -> PyDataFusionResult<Volatility> {
96    Ok(match value {
97        "immutable" => Volatility::Immutable,
98        "stable" => Volatility::Stable,
99        "volatile" => Volatility::Volatile,
100        value => {
101            return Err(PyDataFusionError::Common(format!(
102                "Unsupported volatility type: `{value}`, supported \
103                 values are: immutable, stable and volatile."
104            )))
105        }
106    })
107}
108
109pub(crate) fn validate_pycapsule(capsule: &Bound<PyCapsule>, name: &str) -> PyResult<()> {
110    let capsule_name = capsule.name()?;
111    if capsule_name.is_none() {
112        return Err(PyValueError::new_err(format!(
113            "Expected {name} PyCapsule to have name set."
114        )));
115    }
116
117    let capsule_name = capsule_name.unwrap().to_str()?;
118    if capsule_name != name {
119        return Err(PyValueError::new_err(format!(
120            "Expected name '{name}' in PyCapsule, instead got '{capsule_name}'"
121        )));
122    }
123
124    Ok(())
125}
126
127pub(crate) fn table_provider_from_pycapsule(
128    obj: &Bound<PyAny>,
129) -> PyResult<Option<Arc<dyn TableProvider>>> {
130    if obj.hasattr("__datafusion_table_provider__")? {
131        let capsule = obj.getattr("__datafusion_table_provider__")?.call0()?;
132        let capsule = capsule.downcast::<PyCapsule>().map_err(py_datafusion_err)?;
133        validate_pycapsule(capsule, "datafusion_table_provider")?;
134
135        let provider = unsafe { capsule.reference::<FFI_TableProvider>() };
136        let provider: ForeignTableProvider = provider.into();
137
138        Ok(Some(Arc::new(provider)))
139    } else {
140        Ok(None)
141    }
142}
143
144pub(crate) fn py_obj_to_scalar_value(py: Python, obj: PyObject) -> PyResult<ScalarValue> {
145    // convert Python object to PyScalarValue to ScalarValue
146
147    let pa = py.import("pyarrow")?;
148
149    // Convert Python object to PyArrow scalar
150    let scalar = pa.call_method1("scalar", (obj,))?;
151
152    // Convert PyArrow scalar to PyScalarValue
153    let py_scalar = PyScalarValue::extract_bound(scalar.as_ref())
154        .map_err(|e| PyValueError::new_err(format!("Failed to extract PyScalarValue: {e}")))?;
155
156    // Convert PyScalarValue to ScalarValue
157    Ok(py_scalar.into())
158}