-
Notifications
You must be signed in to change notification settings - Fork 18
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Extensions API for third-party table/catalog providers #43
base: main
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Large diffs are not rendered by default.
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,27 @@ | ||
import ray | ||
|
||
import datafusion_ray | ||
from datafusion_ray import DatafusionRayContext | ||
|
||
## Prerequisites: | ||
## $ brew install roapi | ||
## $ roapi --table taxi=https://d37ci6vzurychx.cloudfront.net/trip-data/yellow_tripdata_2024-01.parquet | ||
## $ maturin develop --features flight-sql-tables | ||
|
||
ray.init() | ||
|
||
ctx = datafusion_ray.extended_session_context({}) | ||
|
||
ray_ctx = DatafusionRayContext(ctx) | ||
|
||
ray_ctx.sql(""" | ||
CREATE EXTERNAL TABLE trip_data | ||
STORED AS FLIGHT_SQL | ||
LOCATION 'http://localhost:32010' | ||
OPTIONS ( | ||
'flight.sql.query' 'SELECT * FROM taxi LIMIT 25' | ||
) | ||
""") | ||
|
||
df = ray_ctx.sql("SELECT tpep_pickup_datetime FROM trip_data LIMIT 10") | ||
print(df.to_pandas()) |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,126 @@ | ||
use async_trait::async_trait; | ||
use datafusion::common::DataFusionError; | ||
use datafusion::common::Result; | ||
use datafusion::execution::FunctionRegistry; | ||
use datafusion::physical_plan::ExecutionPlan; | ||
use datafusion::prelude::SessionContext; | ||
use datafusion_proto::physical_plan::PhysicalExtensionCodec; | ||
use datafusion_python::context::PySessionContext; | ||
use datafusion_python::utils::wait_for_future; | ||
use pyo3::{pyfunction, PyResult, Python}; | ||
use std::collections::HashMap; | ||
use std::fmt::Debug; | ||
use std::sync::{Arc, OnceLock}; | ||
|
||
mod built_in; | ||
|
||
#[cfg(feature = "flight-sql-tables")] | ||
mod flight; | ||
|
||
/// Creates a datafusion session context preconfigured with the enabled extensions | ||
/// that will register additional table providers, catalogs etc. | ||
/// If no extensions are required, the plain `datafusion.SessionContext()` will work just fine. | ||
/// # Arguments | ||
/// * `settings` - dictionary containing extension-specific key/value config options | ||
#[pyfunction] | ||
pub fn extended_session_context( | ||
settings: HashMap<String, String>, | ||
py: Python, | ||
) -> PyResult<PySessionContext> { | ||
let future_context = Extensions::session_context(&settings); | ||
let ctx = wait_for_future(py, future_context)?; | ||
Ok(ctx.into()) | ||
} | ||
|
||
/// Allows third party table/catalog providers, object stores, etc. | ||
/// to be registered with the DataFusion context. | ||
#[async_trait] | ||
trait Extension: Debug + Send + Sync + 'static { | ||
/// SessionContext initialization, using the provided key/value settings if needed. | ||
/// Declared async to allow implementers to perform network or other I/O operations. | ||
async fn init(&self, ctx: &SessionContext, settings: &HashMap<String, String>) -> Result<()> { | ||
let _ = ctx; | ||
let _ = settings; | ||
Ok(()) | ||
} | ||
|
||
/// Codecs for the custom physical plan nodes created by this extension, if any. | ||
fn codecs(&self) -> Vec<Box<dyn PhysicalExtensionCodec>> { | ||
vec![] | ||
} | ||
} | ||
|
||
/// A composite extension registry for enabled extensions. | ||
#[derive(Debug)] | ||
pub(crate) struct Extensions(Box<[Box<dyn Extension>]>); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why not There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's a bit lighter (hardly relevant in this scenario) and it's immutable by design (arguably also irrelevant, though I personally prefer it when I can choose). I think I got this habit after watching this video. :) |
||
|
||
#[async_trait] | ||
impl Extension for Extensions { | ||
async fn init(&self, ctx: &SessionContext, settings: &HashMap<String, String>) -> Result<()> { | ||
for ext in &self.0 { | ||
ext.init(ctx, settings).await?; | ||
} | ||
Ok(()) | ||
} | ||
|
||
fn codecs(&self) -> Vec<Box<dyn PhysicalExtensionCodec>> { | ||
self.0.iter().flat_map(|ext| ext.codecs()).collect() | ||
} | ||
} | ||
|
||
impl Extensions { | ||
fn new() -> Self { | ||
Self(Box::new([ | ||
Box::new(built_in::DefaultExtension::default()), | ||
#[cfg(feature = "flight-sql-tables")] | ||
Box::new(flight::FlightSqlTables::default()), | ||
])) | ||
} | ||
|
||
fn singleton() -> &'static Self { | ||
static EXTENSIONS: OnceLock<Extensions> = OnceLock::new(); | ||
EXTENSIONS.get_or_init(Self::new) | ||
} | ||
|
||
pub(crate) async fn session_context( | ||
settings: &HashMap<String, String>, | ||
) -> Result<SessionContext> { | ||
let ctx = SessionContext::new(); | ||
Self::singleton().init(&ctx, settings).await?; | ||
Ok(ctx) | ||
} | ||
|
||
pub(crate) fn codec() -> &'static CompositeCodec { | ||
static COMPOSITE_CODEC: OnceLock<CompositeCodec> = OnceLock::new(); | ||
COMPOSITE_CODEC.get_or_init(|| CompositeCodec(Extensions::singleton().codecs().into())) | ||
} | ||
} | ||
|
||
/// For both encoding and decoding, tries all the registered extension codecs and returns the first successful result. | ||
#[derive(Debug)] | ||
pub(crate) struct CompositeCodec(Box<[Box<dyn PhysicalExtensionCodec>]>); | ||
|
||
impl PhysicalExtensionCodec for CompositeCodec { | ||
fn try_decode( | ||
&self, | ||
buf: &[u8], | ||
inputs: &[Arc<dyn ExecutionPlan>], | ||
registry: &dyn FunctionRegistry, | ||
) -> Result<Arc<dyn ExecutionPlan>> { | ||
self.0 | ||
.iter() | ||
.filter_map(|codec| codec.try_decode(buf, inputs, registry).ok()) | ||
.next() | ||
.ok_or_else(|| DataFusionError::Execution("No compatible codec found".into())) | ||
} | ||
|
||
fn try_encode(&self, node: Arc<dyn ExecutionPlan>, buf: &mut Vec<u8>) -> Result<()> { | ||
self.0 | ||
.iter() | ||
.filter_map(|codec| codec.try_encode(node.clone(), buf).ok()) | ||
.next() | ||
.ok_or_else(|| { | ||
DataFusionError::Execution(format!("No compatible codec found for {}", node.name())) | ||
}) | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,14 @@ | ||
use crate::ext::Extension; | ||
use crate::shuffle::ShuffleCodec; | ||
use async_trait::async_trait; | ||
use datafusion_proto::physical_plan::PhysicalExtensionCodec; | ||
|
||
#[derive(Debug, Default)] | ||
pub(super) struct DefaultExtension {} | ||
|
||
#[async_trait] | ||
impl Extension for DefaultExtension { | ||
fn codecs(&self) -> Vec<Box<dyn PhysicalExtensionCodec>> { | ||
vec![Box::new(ShuffleCodec {})] | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,33 @@ | ||
use crate::ext::Extension; | ||
use async_trait::async_trait; | ||
use datafusion::prelude::SessionContext; | ||
use datafusion_proto::physical_plan::PhysicalExtensionCodec; | ||
use datafusion_table_providers::flight::codec::FlightPhysicalCodec; | ||
use datafusion_table_providers::flight::sql::FlightSqlDriver; | ||
use datafusion_table_providers::flight::FlightTableFactory; | ||
use std::collections::HashMap; | ||
use std::sync::Arc; | ||
|
||
#[derive(Debug, Default)] | ||
pub(super) struct FlightSqlTables {} | ||
|
||
#[async_trait] | ||
impl Extension for FlightSqlTables { | ||
async fn init( | ||
&self, | ||
ctx: &SessionContext, | ||
_settings: &HashMap<String, String>, | ||
) -> datafusion::common::Result<()> { | ||
ctx.state_ref().write().table_factories_mut().insert( | ||
"FLIGHT_SQL".into(), | ||
Arc::new(FlightTableFactory::new( | ||
Arc::new(FlightSqlDriver::default()), | ||
)), | ||
); | ||
Ok(()) | ||
} | ||
|
||
fn codecs(&self) -> Vec<Box<dyn PhysicalExtensionCodec>> { | ||
vec![Box::new(FlightPhysicalCodec::default())] | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
One of my goals was to remove the
datafusion-python
dependency fromdatafusion-ray
so that you wouldn't have hard requirements about having the exact same version between the two. It can be worse in that you also have to have the same compiler for both. Now fordatafusion-ray
we may be able to get away with it for official releases since we control the build pipeline for both. This does place a restriction on end users in that they have to make sure they keep these versions synced on their machine. In my opinion it would be better to lean on things like the FFI interface that is coming indatafusion-python
43.0.0. I know that right now doesn't solve the problem of having all extensions, though.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Thanks, @timsaucer !
Makes sense, and I'm entirely onboard with this goal. Which is why I tried to preserve this guarantee by keeping the existing code within the
else
branch here, where no assumption is made about thedatafusion-python
version/compiler.The "embedded"
datafusion-python
dependency is only supposed to be an opt-in alternative for users who decide to call the new function for creating an ABI compatible context preconfigured with the enabled extensions (that's the only way the above downcast can succeed, if I'm not mistaken).So any existing or future code that doesn't switch to using the new
extended_session_context()
function will continue to work without any compatibility restrictions.However, if I somehow failed to preserve this guarantee or if I missed something that introduces any potential risks, please let me know so I'll revisit the approach.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't want to hold up progress, but this feels like a step in the wrong direction. But I also don't have enough time right now to give a better solution. I think I'd want to do something like use the FFI_ExecutionPlan in df43 to share these across packages.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Got it, I'll look into that. Thanks!