Skip to content

Commit 6ef1c5b

Browse files
authored
Improve API docs, README, and examples for configuring context (#321)
1 parent 5c90187 commit 6ef1c5b

File tree

8 files changed

+104
-38
lines changed

8 files changed

+104
-38
lines changed

README.md

+33
Original file line numberDiff line numberDiff line change
@@ -86,6 +86,39 @@ This produces the following chart:
8686

8787
![Chart](examples/chart.png)
8888

89+
## Configuration
90+
91+
It is possible to configure runtime (memory and disk settings) and configuration settings when creating a context.
92+
93+
```python
94+
runtime = (
95+
RuntimeConfig()
96+
.with_disk_manager_os()
97+
.with_fair_spill_pool(10000000)
98+
)
99+
config = (
100+
SessionConfig()
101+
.with_create_default_catalog_and_schema(True)
102+
.with_default_catalog_and_schema("foo", "bar")
103+
.with_target_partitions(8)
104+
.with_information_schema(True)
105+
.with_repartition_joins(False)
106+
.with_repartition_aggregations(False)
107+
.with_repartition_windows(False)
108+
.with_parquet_pruning(False)
109+
.set("datafusion.execution.parquet.pushdown_filters", "true")
110+
)
111+
ctx = SessionContext(config, runtime)
112+
```
113+
114+
Refer to the [API documentation](https://arrow.apache.org/datafusion-python/#api-reference) for more information.
115+
116+
Printing the context will show the current configuration settings.
117+
118+
```python
119+
print(ctx)
120+
```
121+
89122
## More Examples
90123

91124
See [examples](examples/README.md) for more information.

dev/release/rat_exclude_files.txt

+1-1
Original file line numberDiff line numberDiff line change
@@ -42,4 +42,4 @@ Cargo.lock
4242
.history
4343
*rat.txt
4444
*/.git
45-
docs.yaml
45+
.github/*

docs/README.md

+2-2
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@
2020
# DataFusion Documentation
2121

2222
This folder contains the source content of the [python api](./source/api).
23-
This is published to https://arrow.apache.org/datafusion-python/ by a GitHub action
23+
This is published to https://arrow.apache.org/datafusion-python/ by a GitHub action
2424
when changes are merged to the main branch.
2525

2626
## Dependencies
@@ -61,4 +61,4 @@ version of the docs, follow these steps:
6161
- `cp -rT ./build/html/ ../../arrow-site/datafusion/` (doesn't work on mac)
6262
- `rsync -avzr ./build/html/ ../../arrow-site/datafusion/`
6363

64-
5. Commit changes in `arrow-site` and send a PR.
64+
5. Commit changes in `arrow-site` and send a PR.

docs/source/api/config.rst

-27
This file was deleted.

docs/source/api/execution_context.rst

+3-1
Original file line numberDiff line numberDiff line change
@@ -19,9 +19,11 @@
1919
.. currentmodule:: datafusion
2020

2121
SessionContext
22-
================
22+
==============
2323

2424
.. autosummary::
2525
:toctree: ../generated/
2626

27+
SessionConfig
28+
RuntimeConfig
2729
SessionContext

examples/README.md

+4
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,10 @@ Here is a direct link to the file used in the examples:
2727

2828
- https://d37ci6vzurychx.cloudfront.net/trip-data/yellow_tripdata_2021-01.parquet
2929

30+
### Creating a SessionContext
31+
32+
- [Creating a SessionContext](./create-context.py)
33+
3034
### Executing Queries with DataFusion
3135

3236
- [Query a Parquet file using SQL](./sql-parquet.py)

examples/create-context.py

+39
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,39 @@
1+
# Licensed to the Apache Software Foundation (ASF) under one
2+
# or more contributor license agreements. See the NOTICE file
3+
# distributed with this work for additional information
4+
# regarding copyright ownership. The ASF licenses this file
5+
# to you under the Apache License, Version 2.0 (the
6+
# "License"); you may not use this file except in compliance
7+
# with the License. You may obtain a copy of the License at
8+
#
9+
# http://www.apache.org/licenses/LICENSE-2.0
10+
#
11+
# Unless required by applicable law or agreed to in writing,
12+
# software distributed under the License is distributed on an
13+
# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14+
# KIND, either express or implied. See the License for the
15+
# specific language governing permissions and limitations
16+
# under the License.
17+
18+
from datafusion import RuntimeConfig, SessionConfig, SessionContext
19+
20+
# create a session context with default settings
21+
ctx = SessionContext()
22+
print(ctx)
23+
24+
# create a session context with explicit runtime and config settings
25+
runtime = RuntimeConfig().with_disk_manager_os().with_fair_spill_pool(10000000)
26+
config = (
27+
SessionConfig()
28+
.with_create_default_catalog_and_schema(True)
29+
.with_default_catalog_and_schema("foo", "bar")
30+
.with_target_partitions(8)
31+
.with_information_schema(True)
32+
.with_repartition_joins(False)
33+
.with_repartition_aggregations(False)
34+
.with_repartition_windows(False)
35+
.with_parquet_pruning(False)
36+
.set("datafusion.execution.parquet.pushdown_filters", "true")
37+
)
38+
ctx = SessionContext(config, runtime)
39+
print(ctx)

src/context.rs

+22-7
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,7 @@ use pyo3::types::PyTuple;
5555
use tokio::runtime::Runtime;
5656
use tokio::task::JoinHandle;
5757

58+
/// Configuration options for a SessionContext
5859
#[pyclass(name = "SessionConfig", module = "datafusion", subclass, unsendable)]
5960
#[derive(Clone, Default)]
6061
pub(crate) struct PySessionConfig {
@@ -141,8 +142,13 @@ impl PySessionConfig {
141142
fn with_parquet_pruning(&self, enabled: bool) -> Self {
142143
Self::from(self.config.clone().with_parquet_pruning(enabled))
143144
}
145+
146+
fn set(&self, key: &str, value: &str) -> Self {
147+
Self::from(self.config.clone().set_str(key, value))
148+
}
144149
}
145150

151+
/// Runtime options for a SessionContext
146152
#[pyclass(name = "RuntimeConfig", module = "datafusion", subclass, unsendable)]
147153
#[derive(Clone)]
148154
pub(crate) struct PyRuntimeConfig {
@@ -549,8 +555,8 @@ impl PySessionContext {
549555
Ok(PyDataFrame::new(self.ctx.read_empty()?))
550556
}
551557

552-
fn session_id(&self) -> PyResult<String> {
553-
Ok(self.ctx.session_id())
558+
fn session_id(&self) -> String {
559+
self.ctx.session_id()
554560
}
555561

556562
#[allow(clippy::too_many_arguments)]
@@ -684,11 +690,20 @@ impl PySessionContext {
684690
}
685691

686692
fn __repr__(&self) -> PyResult<String> {
687-
let id = self.session_id();
688-
match id {
689-
Ok(value) => Ok(format!("SessionContext(session_id={value})")),
690-
Err(err) => Ok(format!("Error: {:?}", err.to_string())),
691-
}
693+
let config = self.ctx.copied_config();
694+
let mut config_entries = config
695+
.options()
696+
.entries()
697+
.iter()
698+
.filter(|e| e.value.is_some())
699+
.map(|e| format!("{} = {}", e.key, e.value.as_ref().unwrap()))
700+
.collect::<Vec<_>>();
701+
config_entries.sort();
702+
Ok(format!(
703+
"SessionContext: id={}; configs=[\n\t{}]",
704+
self.session_id(),
705+
config_entries.join("\n\t")
706+
))
692707
}
693708

694709
/// Execute a partition of an execution plan and return a stream of record batches

0 commit comments

Comments
 (0)