-
Notifications
You must be signed in to change notification settings - Fork 453
Expand file tree
/
Copy pathdatafusion_table_provider.rs
More file actions
129 lines (105 loc) · 4.21 KB
/
datafusion_table_provider.rs
File metadata and controls
129 lines (105 loc) · 4.21 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
// 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::ffi::CString;
use std::sync::Arc;
use datafusion_ffi::table_provider::FFI_TableProvider;
use iceberg::TableIdent;
use iceberg::io::{FileIOBuilder, OpenDalStorageFactory, StorageFactory};
use iceberg::table::StaticTable;
use iceberg_datafusion::table::IcebergStaticTableProvider;
use pyo3::exceptions::PyRuntimeError;
use pyo3::prelude::*;
use pyo3::types::PyCapsule;
use crate::runtime::runtime;
/// Parse the scheme from a URL and return the appropriate StorageFactory.
fn storage_factory_from_path(path: &str) -> PyResult<Arc<dyn StorageFactory>> {
let scheme = path
.split("://")
.next()
.ok_or_else(|| PyRuntimeError::new_err(format!("Invalid path, missing scheme: {path}")))?;
let factory: Arc<dyn StorageFactory> = match scheme {
"file" | "" => Arc::new(OpenDalStorageFactory::Fs),
"s3" | "s3a" => Arc::new(OpenDalStorageFactory::S3 {
customized_credential_load: None,
}),
"memory" => Arc::new(OpenDalStorageFactory::Memory),
_ => {
return Err(PyRuntimeError::new_err(format!(
"Unsupported storage scheme: {scheme}"
)));
}
};
Ok(factory)
}
#[pyclass(name = "IcebergDataFusionTable")]
pub struct PyIcebergDataFusionTable {
inner: Arc<IcebergStaticTableProvider>,
}
#[pymethods]
impl PyIcebergDataFusionTable {
#[new]
fn new(
identifier: Vec<String>,
metadata_location: String,
file_io_properties: Option<HashMap<String, String>>,
) -> PyResult<Self> {
let runtime = runtime();
let provider = runtime.block_on(async {
let table_ident = TableIdent::from_strs(identifier)
.map_err(|e| PyRuntimeError::new_err(format!("Invalid table identifier: {e}")))?;
let factory = storage_factory_from_path(&metadata_location)?;
let mut builder = FileIOBuilder::new(factory);
if let Some(props) = file_io_properties {
builder = builder.with_props(props);
}
let file_io = builder.build();
let static_table =
StaticTable::from_metadata_file(&metadata_location, table_ident, file_io)
.await
.map_err(|e| {
PyRuntimeError::new_err(format!("Failed to load static table: {e}"))
})?;
let table = static_table.into_table();
IcebergStaticTableProvider::try_new_from_table(table)
.await
.map_err(|e| {
PyRuntimeError::new_err(format!("Failed to create table provider: {e}"))
})
})?;
Ok(Self {
inner: Arc::new(provider),
})
}
fn __datafusion_table_provider__<'py>(
&self,
py: Python<'py>,
) -> PyResult<Bound<'py, PyCapsule>> {
let capsule_name = CString::new("datafusion_table_provider").unwrap();
let ffi_provider = FFI_TableProvider::new(self.inner.clone(), false, Some(runtime()));
PyCapsule::new(py, ffi_provider, Some(capsule_name))
}
}
pub fn register_module(py: Python<'_>, m: &Bound<'_, PyModule>) -> PyResult<()> {
let this = PyModule::new(py, "datafusion")?;
this.add_class::<PyIcebergDataFusionTable>()?;
m.add_submodule(&this)?;
py.import("sys")?
.getattr("modules")?
.set_item("pyiceberg_core.datafusion", this)?;
Ok(())
}