8000 Add session context config args (#14) · nirnayroy/datafusion-python@7993e5d · GitHub
[go: up one dir, main page]

Skip to content

Commit 7993e5d

Browse files
authored
Add session context config args (apache#14)
1 parent b759f01 commit 7993e5d

File tree

2 files changed

+65
-5
lines changed

2 files changed

+65
-5
lines changed

datafusion/tests/test_context.py

Lines changed: 25 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -18,7 +18,31 @@
1818
import pyarrow as pa
1919
import pyarrow.dataset as ds
2020

21-
from datafusion import column, literal
21+
from datafusion import column, literal, SessionContext
22+
import pytest
23+
24+
25+
def test_create_context_no_args():
26+
SessionContext()
27+
28+
29+
def test_create_context_with_all_valid_args():
30+
ctx = SessionContext(
31+
target_partitions=1,
32+
default_catalog="foo",
33+
default_schema="bar",
34+
create_default_catalog_and_schema=True,
35+
information_schema=True,
36+
repartition_joins=False,
37+
repartition_aggregations=False,
38+
repartition_windows=False,
39+
parquet_pruning=False,
40+
)
41+
42+
# verify that at least some of the arguments worked
43+
ctx.catalog("foo").database("bar")
44+
with pytest.raises(KeyError):
45+
ctx.catalog("datafusion")
2246

2347

2448
def test_register_record_batches(ctx):

src/context.rs

Lines changed: 40 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ use datafusion::arrow::datatypes::Schema;
2727
use datafusion::arrow::record_batch::RecordBatch;
2828
use datafusion::datasource::datasource::TableProvider;
2929
use datafusion::datasource::MemTable;
30-
use datafusion::execution::context::SessionContext;
30+
use datafusion::execution::context::{SessionConfig, SessionContext};
3131
use datafusion::prelude::{CsvReadOptions, ParquetReadOptions};
3232

3333
use crate::catalog::{PyCatalog, PyTable};
@@ -47,11 +47,47 @@ pub(crate) struct PySessionContext {
4747

4848
#[pymethods]
4949
impl PySessionContext {
50-
// TODO(kszucs): should expose the configuration options as keyword arguments
50+
#[allow(clippy::too_many_arguments)]
51+
#[args(
52+
default_catalog = "\"datafusion\"",
53+
default_schema = "\"public\"",
54+
create_default_catalog_and_schema = "true",
55+
information_schema = "false",
56+
repartition_joins = "true",
57+
repartition_aggregations = "true",
58+
repartition_windows = "true",
59+
parquet_pruning = "true",
60+
target_partitions = "None"
61+
)]
5162
#[new]
52-
fn new() -> Self {
63+
fn new(
64+
default_catalog: &str,
65+
default_schema: &str,
66+
create_default_catalog_and_schema: bool,
67+
information_schema: bool,
68+
repartition_joins: bool,
69+
repartition_aggregations: bool,
70+
repartition_windows: bool,
71+
parquet_pruning: bool,
72+
target_partitions: Option<usize>,
73+
// TODO: config_options
74+
) -> Self {
75+
let cfg = SessionConfig::new()
76+
.create_default_catalog_and_schema(create_default_catalog_and_schema)
77+
.with_default_catalog_and_schema(default_catalog, default_schema)
78+
.with_information_schema(information_schema)
79+
.with_repartition_joins(repartition_joins)
80+
.with_repartition_aggregations(repartition_aggregations)
81+
.with_repartition_windows(repartition_windows)
82+
.with_parquet_pruning(parquet_pruning);
83+
84+
let cfg_full = match target_partitions {
85+
None => cfg,
86+
Some(x) => cfg.with_target_partitions(x),
87+
};
88+
5389
PySessionContext {
54-
ctx: SessionContext::new(),
90+
ctx: SessionContext::with_config(cfg_full),
5591
}
5692
}
5793

0 commit comments

Comments
 (0)
0