Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
49 changes: 47 additions & 2 deletions crates/core/src/context.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@ use datafusion::arrow::datatypes::{DataType, Schema, SchemaRef};
use datafusion::arrow::pyarrow::PyArrowType;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::catalog::{CatalogProvider, CatalogProviderList, TableProviderFactory};
use datafusion::common::{ScalarValue, TableReference, exec_err};
use datafusion::common::{DFSchema, ScalarValue, TableReference, exec_err};
use datafusion::datasource::file_format::file_compression_type::FileCompressionType;
use datafusion::datasource::file_format::parquet::ParquetFormat;
use datafusion::datasource::listing::{
Expand Down Expand Up @@ -60,7 +60,7 @@ use datafusion_python_util::{
};
use object_store::ObjectStore;
use pyo3::IntoPyObjectExt;
use pyo3::exceptions::{PyKeyError, PyValueError};
use pyo3::exceptions::{PyKeyError, PyRuntimeError, PyValueError};
use pyo3::prelude::*;
use pyo3::types::{PyCapsule, PyDict, PyList, PyTuple};
use url::Url;
Expand All @@ -70,11 +70,13 @@ use crate::catalog::{
PyCatalog, PyCatalogList, RustWrappedPyCatalogProvider, RustWrappedPyCatalogProviderList,
};
use crate::common::data_type::PyScalarValue;
use crate::common::df_schema::PyDFSchema;
use crate::dataframe::PyDataFrame;
use crate::dataset::Dataset;
use crate::errors::{
PyDataFusionError, PyDataFusionResult, from_datafusion_error, py_datafusion_err,
};
use crate::expr::PyExpr;
use crate::expr::sort_expr::PySortExpr;
use crate::options::PyCsvReadOptions;
use crate::physical_plan::PyExecutionPlan;
Expand Down Expand Up @@ -1050,6 +1052,49 @@ impl PySessionContext {
self.ctx.session_id()
}

pub fn session_start_time(&self) -> String {
self.ctx.session_start_time().to_rfc3339()
}

pub fn enable_ident_normalization(&self) -> bool {
self.ctx.enable_ident_normalization()
}

pub fn parse_sql_expr(&self, sql: &str, schema: PyDFSchema) -> PyDataFusionResult<PyExpr> {
let df_schema: DFSchema = schema.into();
Ok(self.ctx.parse_sql_expr(sql, &df_schema)?.into())
}

pub fn execute_logical_plan(
&self,
plan: PyLogicalPlan,
py: Python,
) -> PyDataFusionResult<PyDataFrame> {
let df = wait_for_future(
py,
self.ctx.execute_logical_plan(plan.plan.as_ref().clone()),
)??;
Ok(PyDataFrame::new(df))
}

pub fn refresh_catalogs(&self, py: Python) -> PyDataFusionResult<()> {
wait_for_future(py, self.ctx.refresh_catalogs())??;
Ok(())
}

pub fn remove_optimizer_rule(&self, name: &str) -> bool {
self.ctx.remove_optimizer_rule(name)
}

pub fn table_provider(&self, name: &str, py: Python) -> PyResult<PyTable> {
let provider = wait_for_future(py, self.ctx.table_provider(name))
// Outer error: runtime/async failure
.map_err(|e| PyRuntimeError::new_err(e.to_string()))?
// Inner error: table not found
.map_err(|e| PyKeyError::new_err(e.to_string()))?;
Ok(PyTable { table: provider })
}

#[allow(clippy::too_many_arguments)]
#[pyo3(signature = (path, schema=None, schema_infer_max_records=1000, file_extension=".json", table_partition_cols=vec![], file_compression_type=None))]
pub fn read_json(
Expand Down
118 changes: 117 additions & 1 deletion python/datafusion/context.py
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,8 @@
import polars as pl # type: ignore[import]

from datafusion.catalog import CatalogProvider, Table
from datafusion.expr import SortKey
from datafusion.common import DFSchema
from datafusion.expr import Expr, SortKey
from datafusion.plan import ExecutionPlan, LogicalPlan
from datafusion.user_defined import (
AggregateUDF,
Expand Down Expand Up @@ -1141,6 +1142,121 @@ def session_id(self) -> str:
"""Return an id that uniquely identifies this :py:class:`SessionContext`."""
return self.ctx.session_id()

def session_start_time(self) -> str:
"""Return the session start time as an RFC 3339 formatted string.

Examples:
>>> ctx = SessionContext()
>>> ctx.session_start_time() # doctest: +SKIP
'2026-01-01T12:34:56.123456789+00:00'
"""
return self.ctx.session_start_time()

def enable_ident_normalization(self) -> bool:
"""Return whether identifier normalization (lowercasing) is enabled.

Examples:
>>> ctx = SessionContext()
>>> ctx.enable_ident_normalization()
True
"""
return self.ctx.enable_ident_normalization()

def parse_sql_expr(self, sql: str, schema: DFSchema) -> Expr:
"""Parse a SQL expression string into a logical expression.

Args:
sql: SQL expression string.
schema: Schema to use for resolving column references.

Returns:
Parsed expression.

Examples:
>>> from datafusion.common import DFSchema
>>> ctx = SessionContext()
>>> schema = DFSchema.empty()
>>> ctx.parse_sql_expr("1 + 2", schema=schema)
Expr(Int64(1) + Int64(2))
"""
from datafusion.expr import Expr # noqa: PLC0415

return Expr(self.ctx.parse_sql_expr(sql, schema))
Comment on lines +1182 to +1184
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we could remove the import and the wrapping with Expr.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If we do that then we get the unwrapped inner PyExpr, which wouldn't be usable later on.


def execute_logical_plan(self, plan: LogicalPlan) -> DataFrame:
"""Execute a :py:class:`~datafusion.plan.LogicalPlan` and return a DataFrame.

Args:
plan: Logical plan to execute.

Returns:
DataFrame resulting from the execution.

Examples:
>>> ctx = SessionContext()
>>> df = ctx.from_pydict({"a": [1, 2, 3]})
>>> plan = df.logical_plan()
>>> df2 = ctx.execute_logical_plan(plan)
>>> df2.collect()[0].column(0)
<pyarrow.lib.Int64Array object at ...>
[
1,
2,
3
]
"""
return DataFrame(self.ctx.execute_logical_plan(plan._raw_plan))

def refresh_catalogs(self) -> None:
"""Refresh catalog metadata.

Examples:
>>> ctx = SessionContext()
>>> ctx.refresh_catalogs()
"""
self.ctx.refresh_catalogs()

def remove_optimizer_rule(self, name: str) -> bool:
"""Remove an optimizer rule by name.

Args:
name: Name of the optimizer rule to remove.

Returns:
True if a rule with the given name was found and removed.

Examples:
>>> ctx = SessionContext()
>>> ctx.remove_optimizer_rule("nonexistent_rule")
False
"""
return self.ctx.remove_optimizer_rule(name)

def table_provider(self, name: str) -> Table:
"""Return the :py:class:`~datafusion.catalog.Table` for the given table name.

Args:
name: Name of the table.

Returns:
The table provider.

Raises:
KeyError: If the table is not found.

Examples:
>>> import pyarrow as pa
>>> ctx = SessionContext()
>>> batch = pa.RecordBatch.from_pydict({"x": [1, 2]})
>>> ctx.register_record_batches("my_table", [[batch]])
>>> tbl = ctx.table_provider("my_table")
>>> tbl.schema
x: int64
"""
from datafusion.catalog import Table # noqa: PLC0415

return Table(self.ctx.table_provider(name))
Comment on lines +1256 to +1258
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also here I think we can remove the Table and the import.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same, I think we need the wrapper.


def read_json(
self,
path: str | pathlib.Path,
Expand Down
55 changes: 55 additions & 0 deletions python/tests/test_context.py
Original file line number Diff line number Diff line change
Expand Up @@ -551,6 +551,61 @@ def test_table_not_found(ctx):
ctx.table(f"not-found-{uuid4()}")


def test_session_start_time(ctx):
import datetime
import re

st = ctx.session_start_time()
assert isinstance(st, str)
# Truncate nanoseconds to microseconds for Python 3.10 compat
st = re.sub(r"(\.\d{6})\d+", r"\1", st)
dt = datetime.datetime.fromisoformat(st)
assert dt.isoformat()


def test_enable_ident_normalization(ctx):
assert ctx.enable_ident_normalization() is True
ctx.sql("SET datafusion.sql_parser.enable_ident_normalization = false")
assert ctx.enable_ident_normalization() is False


def test_parse_sql_expr(ctx):
from datafusion.common import DFSchema

schema = DFSchema.empty()
expr = ctx.parse_sql_expr("1 + 2", schema)
assert str(expr) == "Expr(Int64(1) + Int64(2))"


def test_execute_logical_plan(ctx):
df = ctx.from_pydict({"a": [1, 2, 3]})
plan = df.logical_plan()
df2 = ctx.execute_logical_plan(plan)
result = df2.collect()
assert result[0].column(0) == pa.array([1, 2, 3])


def test_refresh_catalogs(ctx):
ctx.refresh_catalogs()


def test_remove_optimizer_rule(ctx):
assert ctx.remove_optimizer_rule("push_down_filter") is True
assert ctx.remove_optimizer_rule("nonexistent_rule") is False


def test_table_provider(ctx):
batch = pa.RecordBatch.from_pydict({"x": [10, 20, 30]})
ctx.register_record_batches("provider_test", [[batch]])
tbl = ctx.table_provider("provider_test")
assert tbl.schema == pa.schema([("x", pa.int64())])


def test_table_provider_not_found(ctx):
with pytest.raises(KeyError):
ctx.table_provider("nonexistent_table")


def test_read_json(ctx):
path = pathlib.Path(__file__).parent.resolve()

Expand Down
Loading