diff --git a/Cargo.lock b/Cargo.lock index 1053b248..434b9c33 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4832,6 +4832,7 @@ dependencies = [ "parking_lot", "rstest", "sedona-common", + "sedona-datasource", "sedona-expr", "sedona-functions", "sedona-geo", @@ -4908,6 +4909,7 @@ dependencies = [ "datafusion-physical-plan", "futures", "object_store", + "regex", "sedona-common", "sedona-expr", "sedona-schema", @@ -5309,6 +5311,7 @@ dependencies = [ "pyo3", "sedona", "sedona-adbc", + "sedona-datasource", "sedona-expr", "sedona-geoparquet", "sedona-proj", diff --git a/python/sedonadb/Cargo.toml b/python/sedonadb/Cargo.toml index 939a48e4..af6e7c78 100644 --- a/python/sedonadb/Cargo.toml +++ b/python/sedonadb/Cargo.toml @@ -43,6 +43,7 @@ pyo3 = { version = "0.25.1" } sedona = { path = "../../rust/sedona" } sedona-adbc = { path = "../../rust/sedona-adbc" } sedona-expr = { path = "../../rust/sedona-expr" } +sedona-datasource = { path = "../../rust/sedona-datasource" } sedona-geoparquet = { path = "../../rust/sedona-geoparquet" } sedona-schema = { path = "../../rust/sedona-schema" } sedona-proj = { path = "../../c/sedona-proj", default-features = false } diff --git a/python/sedonadb/python/sedonadb/context.py b/python/sedonadb/python/sedonadb/context.py index f1c48273..b611605d 100644 --- a/python/sedonadb/python/sedonadb/context.py +++ b/python/sedonadb/python/sedonadb/context.py @@ -152,6 +152,26 @@ def read_parquet( self.options, ) + def read_ogr( + self, + table_paths: Union[str, Path, Iterable[str]], + options: Optional[Dict[str, Any]] = None, + ) -> DataFrame: + from sedonadb.datasource import PyogrioFormatSpec + + if isinstance(table_paths, (str, Path)): + table_paths = [table_paths] + + spec = PyogrioFormatSpec() + if options is not None: + spec = spec.with_options(options) + + return DataFrame( + self._impl, + self._impl.read_external_format(spec, [str(path) for path in table_paths], False), + self.options, + ) + def sql(self, sql: str) -> DataFrame: """Create a [DataFrame][sedonadb.dataframe.DataFrame] by executing SQL diff --git a/python/sedonadb/python/sedonadb/datasource.py b/python/sedonadb/python/sedonadb/datasource.py new file mode 100644 index 00000000..f7ffa53e --- /dev/null +++ b/python/sedonadb/python/sedonadb/datasource.py @@ -0,0 +1,98 @@ +# 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. + +from sedonadb._lib import PyExternalFormat + + +class ExternalFormatSpec: + def clone(self): + raise NotImplementedError() + + @property + def extension(self): + return "" + + def with_options(self, options): + raise NotImplementedError( + f"key/value options not supported by {type(self).__name__}" + ) + + def open_reader(self, args): + raise NotImplementedError() + + def infer_schema(self, object): + raise NotImplementedError() + + def __sedona_external_format__(self): + return PyExternalFormat(self) + + +class PyogrioFormatSpec(ExternalFormatSpec): + def __init__(self, extension=""): + import pyogrio.raw + + self._raw = pyogrio.raw + self._extension = extension + self._options = {} + + def clone(self): + cloned = type(self)(self.extension) + cloned._options.update(self._options) + return cloned + + def with_options(self, options): + cloned = self.clone() + cloned._options.update(options) + return cloned + + @property + def extension(self) -> str: + return self._extension + + def open_reader(self, args): + url = args.src.to_url() + if url is None: + raise ValueError(f"Can't convert {args.src} to OGR-openable object") + + if url.startswith("http://") or url.startswith("https://"): + ogr_src = f"/vsicurl/{url}" + elif url.startswith("file://"): + ogr_src = url.removeprefix("file://") + else: + raise ValueError(f"Can't open {url} with OGR") + + if args.is_projected(): + file_names = args.file_schema.names + columns = [file_names[i] for i in args.file_projection] + else: + columns = None + + # TODO: Column order is not respected here, so we still need a utility to + # ensure match with the projected file schema + PyogrioReaderShelter(self._raw.ogr_open_arrow(ogr_src, {}, columns=columns)) + + +class PyogrioReaderShelter: + def __init__(self, inner): + self._inner = inner + self._meta, self._reader = self._inner.__enter__() + + def __del__(self): + self._inner.__exit__(None, None, None) + + def __arrow_c_stream__(self, requested_schema=None): + return self._reader.__arrow_c_stream__() diff --git a/python/sedonadb/src/context.rs b/python/sedonadb/src/context.rs index 4c480484..67ad8dcc 100644 --- a/python/sedonadb/src/context.rs +++ b/python/sedonadb/src/context.rs @@ -23,6 +23,7 @@ use tokio::runtime::Runtime; use crate::{ dataframe::InternalDataFrame, + datasource::PyExternalFormat, error::PySedonaError, import_from::{import_ffi_scalar_udf, import_table_provider_from_any}, runtime::wait_for_future, @@ -107,6 +108,26 @@ impl InternalContext { Ok(InternalDataFrame::new(df, self.runtime.clone())) } + pub fn read_external_format<'py>( + &self, + py: Python<'py>, + format_spec: Bound, + table_paths: Vec, + check_extension: bool, + ) -> Result { + let spec = format_spec + .call_method0("__sedona_external_format__")? + .extract::()?; + let df = wait_for_future( + py, + &self.runtime, + self.inner + .read_external_format(Arc::new(spec), table_paths, None, check_extension), + )??; + + Ok(InternalDataFrame::new(df, self.runtime.clone())) + } + pub fn sql<'py>( &self, py: Python<'py>, diff --git a/python/sedonadb/src/datasource.rs b/python/sedonadb/src/datasource.rs new file mode 100644 index 00000000..4a54f8b4 --- /dev/null +++ b/python/sedonadb/src/datasource.rs @@ -0,0 +1,250 @@ +// 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, sync::Arc}; + +use arrow_array::{RecordBatch, RecordBatchReader}; +use arrow_schema::{ArrowError, Schema, SchemaRef}; +use async_trait::async_trait; +use datafusion_common::{DataFusionError, Result}; +use pyo3::{exceptions::PyNotImplementedError, pyclass, pymethods, PyObject, Python}; +use sedona_datasource::spec::{ExternalFormatSpec, Object, OpenReaderArgs}; + +use crate::{ + error::PySedonaError, + import_from::{import_arrow_array_stream, import_arrow_schema}, + schema::PySedonaSchema, +}; + +#[pyclass] +#[derive(Debug)] +pub struct PyExternalFormat { + extension: String, + py_spec: PyObject, +} + +impl Clone for PyExternalFormat { + fn clone(&self) -> Self { + Python::with_gil(|py| Self { + extension: self.extension.clone(), + py_spec: self.py_spec.clone_ref(py), + }) + } +} + +impl PyExternalFormat { + fn with_options_impl<'py>( + &self, + py: Python<'py>, + options: &HashMap, + ) -> Result { + let new_py_spec = self + .py_spec + .call_method(py, "with_options", (options.clone(),), None)?; + let new_extension = new_py_spec + .getattr(py, "extension")? + .extract::(py)?; + Ok(Self { + extension: new_extension, + py_spec: new_py_spec, + }) + } + + fn infer_schema_impl<'py>( + &self, + py: Python<'py>, + object: &Object, + ) -> Result { + let maybe_schema = self.py_spec.call_method( + py, + "infer_schema", + (PyDataSourceObject { + inner: object.clone(), + },), + None, + ); + + match maybe_schema { + Ok(py_schema) => import_arrow_schema(py_schema.bind(py)), + Err(e) => { + if e.is_instance_of::(py) { + // Fall back on the open_reader implementation + let reader_args = OpenReaderArgs { + src: object.clone(), + batch_size: None, + file_schema: None, + file_projection: None, + filters: vec![], + }; + + let reader = self.open_reader_impl(py, &reader_args)?; + Ok(reader.schema().as_ref().clone()) + } else { + Err(PySedonaError::from(e)) + } + } + } + } + + fn open_reader_impl<'py>( + &self, + py: Python<'py>, + args: &OpenReaderArgs, + ) -> Result, PySedonaError> { + let reader_obj = self.py_spec.call_method( + py, + "open_reader", + (PyOpenReaderArgs { + inner: args.clone(), + },), + None, + )?; + + let reader = import_arrow_array_stream(py, reader_obj.bind(py), None)?; + let wrapped_reader = WrappedRecordBatchReader { + inner: reader, + shelter: Some(reader_obj), + }; + Ok(Box::new(wrapped_reader)) + } +} + +#[pymethods] +impl PyExternalFormat { + #[new] + fn new<'py>(py: Python<'py>, py_spec: PyObject) -> Result { + let extension = py_spec.getattr(py, "extension")?.extract::(py)?; + Ok(Self { extension, py_spec }) + } +} + +#[async_trait] +impl ExternalFormatSpec for PyExternalFormat { + fn extension(&self) -> &str { + &self.extension + } + + fn with_options( + &self, + options: &HashMap, + ) -> Result> { + let new_external_format = Python::with_gil(|py| self.with_options_impl(py, options)) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + Ok(Arc::new(new_external_format)) + } + + async fn infer_schema(&self, location: &Object) -> Result { + let schema = Python::with_gil(|py| self.infer_schema_impl(py, location)) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + + Ok(schema) + } + + async fn open_reader( + &self, + args: &OpenReaderArgs, + ) -> Result> { + let reader = Python::with_gil(|py| self.open_reader_impl(py, args)) + .map_err(|e| DataFusionError::External(Box::new(e)))?; + + Ok(reader) + } +} + +#[pyclass] +#[derive(Clone, Debug)] +pub struct PyDataSourceObject { + pub inner: Object, +} + +#[pymethods] +impl PyDataSourceObject { + fn to_url(&self) -> Option { + self.inner.to_url_string() + } +} + +#[pyclass] +#[derive(Clone, Debug)] +pub struct PyOpenReaderArgs { + pub inner: OpenReaderArgs, +} + +#[pymethods] +impl PyOpenReaderArgs { + #[getter] + fn src(&self) -> PyDataSourceObject { + PyDataSourceObject { + inner: self.inner.src.clone(), + } + } + + #[getter] + fn batch_size(&self) -> Option { + self.inner.batch_size + } + + #[getter] + fn file_schema(&self) -> Option { + self.inner + .file_schema + .as_ref() + .map(|schema| PySedonaSchema::new(schema.as_ref().clone())) + } + + #[getter] + fn file_projection(&self) -> Option> { + self.inner.file_projection.clone() + } + + fn is_projected(&self) -> Result { + match (&self.inner.file_projection, &self.inner.file_schema) { + (None, None) | (None, Some(_)) => Ok(false), + (Some(projection), Some(schema)) => { + let seq_along_schema = (0..schema.fields().len()).collect::>(); + Ok(&seq_along_schema != projection) + } + (Some(_), None) => Err(PySedonaError::SedonaPython( + "Can't check projection for OpenReaderArgs with no schena".to_string(), + )), + } + } +} + +struct WrappedRecordBatchReader { + pub inner: Box, + pub shelter: Option, +} + +impl RecordBatchReader for WrappedRecordBatchReader { + fn schema(&self) -> SchemaRef { + self.inner.schema() + } +} + +impl Iterator for WrappedRecordBatchReader { + type Item = Result; + + fn next(&mut self) -> Option { + if let Some(item) = self.inner.next() { + Some(item) + } else { + self.shelter = None; + None + } + } +} diff --git a/python/sedonadb/src/lib.rs b/python/sedonadb/src/lib.rs index 62a0caba..b40608c9 100644 --- a/python/sedonadb/src/lib.rs +++ b/python/sedonadb/src/lib.rs @@ -22,6 +22,7 @@ use std::ffi::c_void; mod context; mod dataframe; +mod datasource; mod error; mod import_from; mod reader; @@ -94,6 +95,7 @@ fn _lib(py: Python<'_>, m: &Bound<'_, PyModule>) -> PyResult<()> { m.add_class::()?; m.add_class::()?; + m.add_class::()?; m.add("SedonaError", py.get_type::())?; m.add_class::()?; m.add_class::()?; diff --git a/python/sedonadb/src/schema.rs b/python/sedonadb/src/schema.rs index d261043c..584fb200 100644 --- a/python/sedonadb/src/schema.rs +++ b/python/sedonadb/src/schema.rs @@ -59,6 +59,15 @@ impl PySedonaSchema { #[pymethods] impl PySedonaSchema { + #[getter] + fn names(&self) -> Vec { + self.inner + .fields() + .iter() + .map(|f| f.name().to_string()) + .collect() + } + fn field<'py>( &self, py: Python<'py>, diff --git a/python/sedonadb/tests/test_datasource.py b/python/sedonadb/tests/test_datasource.py new file mode 100644 index 00000000..d7e3d08b --- /dev/null +++ b/python/sedonadb/tests/test_datasource.py @@ -0,0 +1,46 @@ +# 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. + +import tempfile + +import geopandas +import geopandas.testing + + +def test_read_ogr(con): + n = 1024 + series = geopandas.GeoSeries.from_xy( + list(range(n)), list(range(1, n + 1)), crs="EPSG:3857" + ) + gdf = geopandas.GeoDataFrame({"idx": list(range(n)), "wkb_geometry": series}) + gdf = gdf.set_geometry(gdf["wkb_geometry"]) + + with tempfile.TemporaryDirectory() as td: + temp_fgb_path = f"{td}/temp.fgb" + gdf.to_file(temp_fgb_path) + con.read_ogr(temp_fgb_path).to_view("test_fgb", overwrite=True) + + # With no projection + geopandas.testing.assert_geodataframe_equal( + con.sql("SELECT * FROM test_fgb ORDER BY idx").to_pandas(), gdf + ) + + # With only not geometry selected + geopandas.testing.assert_geodataframe_equal( + con.sql("SELECT idx FROM test_fgb ORDER BY idx").to_pandas(), + gdf.filter(["idx"]), + ) diff --git a/rust/sedona-datasource/Cargo.toml b/rust/sedona-datasource/Cargo.toml index de0dfffe..f2273e0c 100644 --- a/rust/sedona-datasource/Cargo.toml +++ b/rust/sedona-datasource/Cargo.toml @@ -45,6 +45,7 @@ datafusion-physical-expr = { workspace = true } datafusion-physical-plan = { workspace = true } futures = { workspace = true } object_store = { workspace = true } +regex = { workspace = true } sedona-common = { path = "../sedona-common" } sedona-expr = { path = "../sedona-expr" } sedona-schema = { path = "../sedona-schema" } diff --git a/rust/sedona-datasource/src/spec.rs b/rust/sedona-datasource/src/spec.rs index a3abf8d6..706dde5d 100644 --- a/rust/sedona-datasource/src/spec.rs +++ b/rust/sedona-datasource/src/spec.rs @@ -26,6 +26,7 @@ use datafusion_common::{Result, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_physical_expr::PhysicalExpr; use object_store::{ObjectMeta, ObjectStore}; +use regex::Regex; /// Simple file format specification /// @@ -182,7 +183,14 @@ impl Object { // GDAL to be able to translate. let object_store_debug = format!("{:?}", self.store).to_lowercase(); if object_store_debug.contains("http") { - Some(format!("https://{}", meta.location)) + let pattern = r#"host: some\(domain\("([A-Za-z0-9.-]+)"\)\)"#; + let re = Regex::new(pattern).ok()?; + if let Some(caps) = re.captures(&object_store_debug) { + caps.get(1) + .map(|host| format!("https://{}/{}", host.as_str(), meta.location)) + } else { + None + } } else if object_store_debug.contains("local") { Some(format!("file:///{}", meta.location)) } else { @@ -190,7 +198,7 @@ impl Object { } } (Some(url), None) => Some(url.to_string()), - (Some(url), Some(meta)) => Some(format!("{url}/{}", meta.location)), + (Some(url), Some(meta)) => Some(format!("{url}{}", meta.location)), (None, None) => None, } } diff --git a/rust/sedona/Cargo.toml b/rust/sedona/Cargo.toml index c7cb95a2..450ab3a9 100644 --- a/rust/sedona/Cargo.toml +++ b/rust/sedona/Cargo.toml @@ -63,6 +63,7 @@ geo-types = { workspace = true } object_store = { workspace = true } parking_lot = { workspace = true } sedona-common = { path = "../sedona-common" } +sedona-datasource = { path = "../sedona-datasource" } sedona-expr = { path = "../sedona-expr" } sedona-functions = { path = "../sedona-functions" } sedona-geo = { path = "../sedona-geo", optional = true } diff --git a/rust/sedona/src/context.rs b/rust/sedona/src/context.rs index 744dca22..4d26c935 100644 --- a/rust/sedona/src/context.rs +++ b/rust/sedona/src/context.rs @@ -1,3 +1,4 @@ +use std::collections::HashMap; // 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 @@ -40,6 +41,8 @@ use datafusion_expr::sqlparser::dialect::{dialect_from_str, Dialect}; use datafusion_expr::{LogicalPlan, LogicalPlanBuilder, SortExpr}; use parking_lot::Mutex; use sedona_common::option::add_sedona_option_extension; +use sedona_datasource::provider::external_listing_table; +use sedona_datasource::spec::ExternalFormatSpec; use sedona_expr::aggregate_udf::SedonaAccumulatorRef; use sedona_expr::{function_set::FunctionSet, scalar_udf::ScalarKernelRef}; use sedona_geoparquet::options::TableGeoParquetOptions; @@ -251,6 +254,43 @@ impl SedonaContext { self.ctx.read_table(Arc::new(provider)) } + + /// Creates a [`DataFrame`] for reading a [ExternalFormatSpec] + pub async fn read_external_format( + &self, + spec: Arc, + table_paths: P, + options: Option<&HashMap>, + check_extension: bool, + ) -> Result { + let urls = table_paths.to_urls()?; + + // Pre-register object store with our custom options before creating GeoParquetReadOptions + if !urls.is_empty() { + // Extract the table options from GeoParquetReadOptions for object store registration + ensure_object_store_registered_with_options( + &mut self.ctx.state(), + urls[0].as_str(), + options, + ) + .await?; + } + + let provider = if let Some(options) = options { + // Strip the filesystem-based options + let options_without_filesystems = options + .iter() + .filter(|(k, _)| !k.starts_with("gcs.") || !k.starts_with("aws.")) + .map(|(k, v)| (k.clone(), v.clone())) + .collect::>(); + let spec = spec.with_options(&options_without_filesystems)?; + external_listing_table(spec, &self.ctx, urls, check_extension).await? + } else { + external_listing_table(spec, &self.ctx, urls, check_extension).await? + }; + + self.ctx.read_table(Arc::new(provider)) + } } impl Default for SedonaContext {