| // Licensed to the Apache Software Foundation (ASF) under one |
| // or more contributor license agreements. See the NOTICE file |
| // distributed with this work for additional information |
| // regarding copyright ownership. The ASF licenses this file |
| // to you under the Apache License, Version 2.0 (the |
| // "License"); you may not use this file except in compliance |
| // with the License. You may obtain a copy of the License at |
| // |
| // http://www.apache.org/licenses/LICENSE-2.0 |
| // |
| // Unless required by applicable law or agreed to in writing, |
| // software distributed under the License is distributed on an |
| // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| // KIND, either express or implied. See the License for the |
| // specific language governing permissions and limitations |
| // under the License. |
| |
| use std::collections::HashMap; |
| use std::ptr::NonNull; |
| use std::sync::Arc; |
| |
| use datafusion::catalog::CatalogProvider; |
| use datafusion_ffi::catalog_provider::FFI_CatalogProvider; |
| use datafusion_ffi::proto::logical_extension_codec::FFI_LogicalExtensionCodec; |
| use paimon::{CatalogFactory, Options}; |
| use paimon_datafusion::PaimonCatalogProvider; |
| use pyo3::exceptions::PyValueError; |
| use pyo3::prelude::*; |
| use pyo3::types::PyCapsule; |
| |
| use crate::error::to_py_err; |
| use paimon_datafusion::runtime::runtime; |
| |
| fn build_paimon_catalog_provider( |
| catalog_options: HashMap<String, String>, |
| ) -> PyResult<Arc<PaimonCatalogProvider>> { |
| let rt = runtime(); |
| rt.block_on(async { |
| let options = Options::from_map(catalog_options); |
| let catalog = CatalogFactory::create(options).await.map_err(to_py_err)?; |
| Ok::<_, PyErr>(Arc::new(PaimonCatalogProvider::new(catalog))) |
| }) |
| } |
| |
| fn ffi_logical_codec_from_pycapsule(obj: Bound<'_, PyAny>) -> PyResult<FFI_LogicalExtensionCodec> { |
| let attr_name = "__datafusion_logical_extension_codec__"; |
| let capsule = if obj.hasattr(attr_name)? { |
| obj.getattr(attr_name)?.call0()? |
| } else { |
| obj |
| }; |
| |
| let capsule = capsule.cast::<PyCapsule>()?; |
| let expected_name = c"datafusion_logical_extension_codec"; |
| match capsule.name()? { |
| Some(name) if name == expected_name => {} |
| Some(name) => { |
| return Err(PyValueError::new_err(format!( |
| "Expected capsule named {expected_name:?}, got {name:?}" |
| ))); |
| } |
| None => { |
| return Err(PyValueError::new_err(format!( |
| "Expected capsule named {expected_name:?}, got unnamed capsule" |
| ))); |
| } |
| } |
| |
| let data = NonNull::new(capsule.pointer().cast::<FFI_LogicalExtensionCodec>()) |
| .ok_or_else(|| PyValueError::new_err("Null logical extension codec capsule pointer"))?; |
| let codec = unsafe { data.as_ref() }; |
| |
| Ok(codec.clone()) |
| } |
| |
| /// A Paimon catalog exportable to Python DataFusion `SessionContext`. |
| #[pyclass(name = "PaimonCatalog")] |
| pub struct PaimonCatalog { |
| provider: Arc<PaimonCatalogProvider>, |
| } |
| |
| #[pymethods] |
| impl PaimonCatalog { |
| /// Create a Paimon catalog that can be registered into a DataFusion session. |
| #[new] |
| fn new(catalog_options: HashMap<String, String>) -> PyResult<Self> { |
| Ok(Self { |
| provider: build_paimon_catalog_provider(catalog_options)?, |
| }) |
| } |
| |
| /// Export this catalog as a DataFusion catalog provider PyCapsule. |
| fn __datafusion_catalog_provider__<'py>( |
| &self, |
| py: Python<'py>, |
| session: Bound<'py, PyAny>, |
| ) -> PyResult<Bound<'py, PyCapsule>> { |
| let name = cr"datafusion_catalog_provider".into(); |
| let provider = Arc::clone(&self.provider) as Arc<dyn CatalogProvider + Send>; |
| let codec = ffi_logical_codec_from_pycapsule(session)?; |
| let provider = FFI_CatalogProvider::new_with_ffi_codec(provider, Some(runtime()), codec); |
| PyCapsule::new(py, provider, Some(name)) |
| } |
| } |
| |
| pub fn register_module(py: Python<'_>, m: &Bound<'_, PyModule>) -> PyResult<()> { |
| let this = PyModule::new(py, "datafusion")?; |
| this.add_class::<PaimonCatalog>()?; |
| m.add_submodule(&this)?; |
| py.import("sys")? |
| .getattr("modules")? |
| .set_item("pypaimon_rust.datafusion", this)?; |
| Ok(()) |
| } |