You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by ag...@apache.org on 2023/04/12 10:26:58 UTC

[arrow-datafusion-python] branch main updated: Improve API docs, README, and examples for configuring context (#321)

This is an automated email from the ASF dual-hosted git repository.

agrove pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/arrow-datafusion-python.git


The following commit(s) were added to refs/heads/main by this push:
     new 6ef1c5b  Improve API docs, README, and examples for configuring context (#321)
6ef1c5b is described below

commit 6ef1c5b554c452bbae2d25b44308722ccfa8fbf7
Author: Andy Grove <an...@gmail.com>
AuthorDate: Wed Apr 12 04:26:52 2023 -0600

    Improve API docs, README, and examples for configuring context (#321)
---
 README.md                             | 33 +++++++++++++++++++++++++++++
 dev/release/rat_exclude_files.txt     |  2 +-
 docs/README.md                        |  4 ++--
 docs/source/api/config.rst            | 27 ------------------------
 docs/source/api/execution_context.rst |  4 +++-
 examples/README.md                    |  4 ++++
 examples/create-context.py            | 39 +++++++++++++++++++++++++++++++++++
 src/context.rs                        | 29 +++++++++++++++++++-------
 8 files changed, 104 insertions(+), 38 deletions(-)

diff --git a/README.md b/README.md
index 7c29def..506a382 100644
--- a/README.md
+++ b/README.md
@@ -86,6 +86,39 @@ This produces the following chart:
 
 ![Chart](examples/chart.png)
 
+## Configuration
+
+It is possible to configure runtime (memory and disk settings) and configuration settings when creating a context.
+
+```python
+runtime = (
+    RuntimeConfig()
+    .with_disk_manager_os()
+    .with_fair_spill_pool(10000000)
+)
+config = (
+    SessionConfig()
+    .with_create_default_catalog_and_schema(True)
+    .with_default_catalog_and_schema("foo", "bar")
+    .with_target_partitions(8)
+    .with_information_schema(True)
+    .with_repartition_joins(False)
+    .with_repartition_aggregations(False)
+    .with_repartition_windows(False)
+    .with_parquet_pruning(False)
+    .set("datafusion.execution.parquet.pushdown_filters", "true")
+)
+ctx = SessionContext(config, runtime)
+```
+
+Refer to the [API documentation](https://arrow.apache.org/datafusion-python/#api-reference) for more information.
+
+Printing the context will show the current configuration settings.
+
+```python
+print(ctx)
+```
+
 ## More Examples
 
 See [examples](examples/README.md) for more information.
diff --git a/dev/release/rat_exclude_files.txt b/dev/release/rat_exclude_files.txt
index db5379d..c7754f3 100644
--- a/dev/release/rat_exclude_files.txt
+++ b/dev/release/rat_exclude_files.txt
@@ -42,4 +42,4 @@ Cargo.lock
 .history
 *rat.txt
 */.git
-docs.yaml
\ No newline at end of file
+.github/*
\ No newline at end of file
diff --git a/docs/README.md b/docs/README.md
index 6d8b2d5..04f46a9 100644
--- a/docs/README.md
+++ b/docs/README.md
@@ -20,7 +20,7 @@
 # DataFusion Documentation
 
 This folder contains the source content of the [python api](./source/api).
-This is published to https://arrow.apache.org/datafusion-python/ by a GitHub action 
+This is published to https://arrow.apache.org/datafusion-python/ by a GitHub action
 when changes are merged to the main branch.
 
 ## Dependencies
@@ -61,4 +61,4 @@ version of the docs, follow these steps:
 - `cp -rT ./build/html/ ../../arrow-site/datafusion/` (doesn't work on mac)
 - `rsync -avzr ./build/html/ ../../arrow-site/datafusion/`
 
-5. Commit changes in `arrow-site` and send a PR.
\ No newline at end of file
+5. Commit changes in `arrow-site` and send a PR.
diff --git a/docs/source/api/config.rst b/docs/source/api/config.rst
deleted file mode 100644
index df244ae..0000000
--- a/docs/source/api/config.rst
+++ /dev/null
@@ -1,27 +0,0 @@
-.. 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.
-
-.. _api.config:
-.. currentmodule:: datafusion
-
-Config
-=========
-
-.. autosummary::
-   :toctree: ../generated/
-
-   Config
diff --git a/docs/source/api/execution_context.rst b/docs/source/api/execution_context.rst
index 5b7e0f8..a3bda76 100644
--- a/docs/source/api/execution_context.rst
+++ b/docs/source/api/execution_context.rst
@@ -19,9 +19,11 @@
 .. currentmodule:: datafusion
 
 SessionContext
-================
+==============
 
 .. autosummary::
    :toctree: ../generated/
 
+   SessionConfig
+   RuntimeConfig
    SessionContext
diff --git a/examples/README.md b/examples/README.md
index 2c4775e..8240595 100644
--- a/examples/README.md
+++ b/examples/README.md
@@ -27,6 +27,10 @@ Here is a direct link to the file used in the examples:
 
 - https://d37ci6vzurychx.cloudfront.net/trip-data/yellow_tripdata_2021-01.parquet
 
+### Creating a SessionContext
+
+- [Creating a SessionContext](./create-context.py)
+
 ### Executing Queries with DataFusion
 
 - [Query a Parquet file using SQL](./sql-parquet.py)
diff --git a/examples/create-context.py b/examples/create-context.py
new file mode 100644
index 0000000..3184d40
--- /dev/null
+++ b/examples/create-context.py
@@ -0,0 +1,39 @@
+# 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.
+
+from datafusion import RuntimeConfig, SessionConfig, SessionContext
+
+# create a session context with default settings
+ctx = SessionContext()
+print(ctx)
+
+# create a session context with explicit runtime and config settings
+runtime = RuntimeConfig().with_disk_manager_os().with_fair_spill_pool(10000000)
+config = (
+    SessionConfig()
+    .with_create_default_catalog_and_schema(True)
+    .with_default_catalog_and_schema("foo", "bar")
+    .with_target_partitions(8)
+    .with_information_schema(True)
+    .with_repartition_joins(False)
+    .with_repartition_aggregations(False)
+    .with_repartition_windows(False)
+    .with_parquet_pruning(False)
+    .set("datafusion.execution.parquet.pushdown_filters", "true")
+)
+ctx = SessionContext(config, runtime)
+print(ctx)
diff --git a/src/context.rs b/src/context.rs
index 448fa77..0ba1200 100644
--- a/src/context.rs
+++ b/src/context.rs
@@ -55,6 +55,7 @@ use pyo3::types::PyTuple;
 use tokio::runtime::Runtime;
 use tokio::task::JoinHandle;
 
+/// Configuration options for a SessionContext
 #[pyclass(name = "SessionConfig", module = "datafusion", subclass, unsendable)]
 #[derive(Clone, Default)]
 pub(crate) struct PySessionConfig {
@@ -141,8 +142,13 @@ impl PySessionConfig {
     fn with_parquet_pruning(&self, enabled: bool) -> Self {
         Self::from(self.config.clone().with_parquet_pruning(enabled))
     }
+
+    fn set(&self, key: &str, value: &str) -> Self {
+        Self::from(self.config.clone().set_str(key, value))
+    }
 }
 
+/// Runtime options for a SessionContext
 #[pyclass(name = "RuntimeConfig", module = "datafusion", subclass, unsendable)]
 #[derive(Clone)]
 pub(crate) struct PyRuntimeConfig {
@@ -549,8 +555,8 @@ impl PySessionContext {
         Ok(PyDataFrame::new(self.ctx.read_empty()?))
     }
 
-    fn session_id(&self) -> PyResult<String> {
-        Ok(self.ctx.session_id())
+    fn session_id(&self) -> String {
+        self.ctx.session_id()
     }
 
     #[allow(clippy::too_many_arguments)]
@@ -684,11 +690,20 @@ impl PySessionContext {
     }
 
     fn __repr__(&self) -> PyResult<String> {
-        let id = self.session_id();
-        match id {
-            Ok(value) => Ok(format!("SessionContext(session_id={value})")),
-            Err(err) => Ok(format!("Error: {:?}", err.to_string())),
-        }
+        let config = self.ctx.copied_config();
+        let mut config_entries = config
+            .options()
+            .entries()
+            .iter()
+            .filter(|e| e.value.is_some())
+            .map(|e| format!("{} = {}", e.key, e.value.as_ref().unwrap()))
+            .collect::<Vec<_>>();
+        config_entries.sort();
+        Ok(format!(
+            "SessionContext: id={}; configs=[\n\t{}]",
+            self.session_id(),
+            config_entries.join("\n\t")
+        ))
     }
 
     /// Execute a partition of an execution plan and return a stream of record batches