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 613ca34  feature: Create dataframe from pandas, polars, dictionary, 
list or pyarrow Table (#242)
613ca34 is described below

commit 613ca3406b82b9a52bdb949b5d4f40d01f03132b
Author: Dejan Simic <[email protected]>
AuthorDate: Tue Feb 28 18:51:47 2023 +0100

    feature: Create dataframe from pandas, polars, dictionary, list or pyarrow 
Table (#242)
    
    * Implement from_arrow_table(), from_pydict() & from_pylist()
    
    * Refactor functions
    
    * Implement from_pandas() & from_polars()
    
    * Document new import functions
    
    * Fix clippy errors
---
 datafusion/tests/test_context.py | 81 ++++++++++++++++++++++++++++++++++++++++
 examples/export.py               |  9 +----
 examples/import.py               | 58 ++++++++++++++++++++++++++++
 src/context.rs                   | 75 +++++++++++++++++++++++++++++++++++++
 4 files changed, 216 insertions(+), 7 deletions(-)

diff --git a/datafusion/tests/test_context.py b/datafusion/tests/test_context.py
index efa2ede..0cdf380 100644
--- a/datafusion/tests/test_context.py
+++ b/datafusion/tests/test_context.py
@@ -26,6 +26,7 @@ from datafusion import (
     SessionContext,
     SessionConfig,
     RuntimeConfig,
+    DataFrame,
 )
 import pytest
 
@@ -95,6 +96,86 @@ def test_create_dataframe_registers_unique_table_name(ctx):
         assert c in "0123456789abcdef"
 
 
+def test_from_arrow_table(ctx):
+    # create a PyArrow table
+    data = {"a": [1, 2, 3], "b": [4, 5, 6]}
+    table = pa.Table.from_pydict(data)
+
+    # convert to DataFrame
+    df = ctx.from_arrow_table(table)
+    tables = list(ctx.tables())
+
+    assert df
+    assert len(tables) == 1
+    assert type(df) == DataFrame
+    assert set(df.schema().names) == {"a", "b"}
+    assert df.collect()[0].num_rows == 3
+
+
+def test_from_pylist(ctx):
+    # create a dataframe from Python list
+    data = [
+        {"a": 1, "b": 4},
+        {"a": 2, "b": 5},
+        {"a": 3, "b": 6},
+    ]
+
+    df = ctx.from_pylist(data)
+    tables = list(ctx.tables())
+
+    assert df
+    assert len(tables) == 1
+    assert type(df) == DataFrame
+    assert set(df.schema().names) == {"a", "b"}
+    assert df.collect()[0].num_rows == 3
+
+
+def test_from_pydict(ctx):
+    # create a dataframe from Python dictionary
+    data = {"a": [1, 2, 3], "b": [4, 5, 6]}
+
+    df = ctx.from_pydict(data)
+    tables = list(ctx.tables())
+
+    assert df
+    assert len(tables) == 1
+    assert type(df) == DataFrame
+    assert set(df.schema().names) == {"a", "b"}
+    assert df.collect()[0].num_rows == 3
+
+
+def test_from_pandas(ctx):
+    # create a dataframe from pandas dataframe
+    pd = pytest.importorskip("pandas")
+    data = {"a": [1, 2, 3], "b": [4, 5, 6]}
+    pandas_df = pd.DataFrame(data)
+
+    df = ctx.from_pandas(pandas_df)
+    tables = list(ctx.tables())
+
+    assert df
+    assert len(tables) == 1
+    assert type(df) == DataFrame
+    assert set(df.schema().names) == {"a", "b"}
+    assert df.collect()[0].num_rows == 3
+
+
+def test_from_polars(ctx):
+    # create a dataframe from Polars dataframe
+    pd = pytest.importorskip("polars")
+    data = {"a": [1, 2, 3], "b": [4, 5, 6]}
+    polars_df = pd.DataFrame(data)
+
+    df = ctx.from_polars(polars_df)
+    tables = list(ctx.tables())
+
+    assert df
+    assert len(tables) == 1
+    assert type(df) == DataFrame
+    assert set(df.schema().names) == {"a", "b"}
+    assert df.collect()[0].num_rows == 3
+
+
 def test_register_table(ctx, database):
     default = ctx.catalog()
     public = default.database("public")
diff --git a/examples/export.py b/examples/export.py
index b7fe2b6..d179bf3 100644
--- a/examples/export.py
+++ b/examples/export.py
@@ -16,18 +16,13 @@
 # under the License.
 
 import datafusion
-import pyarrow
 
 
 # create a context
 ctx = datafusion.SessionContext()
 
-# create a RecordBatch and a new datafusion DataFrame from it
-batch = pyarrow.RecordBatch.from_arrays(
-    [pyarrow.array([1, 2, 3]), pyarrow.array([4, 5, 6])],
-    names=["a", "b"],
-)
-df = ctx.create_dataframe([[batch]])
+# create a new datafusion DataFrame
+df = ctx.from_pydict({"a": [1, 2, 3], "b": [4, 5, 6]})
 # Dataframe:
 # +---+---+
 # | a | b |
diff --git a/examples/import.py b/examples/import.py
new file mode 100644
index 0000000..a249a1c
--- /dev/null
+++ b/examples/import.py
@@ -0,0 +1,58 @@
+# 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.
+
+import datafusion
+import pyarrow as pa
+import pandas as pd
+import polars as pl
+
+
+# Create a context
+ctx = datafusion.SessionContext()
+
+# Create a datafusion DataFrame from a Python dictionary
+# The dictionary keys represent column names and the dictionary values
+# represent column values
+df = ctx.from_pydict({"a": [1, 2, 3], "b": [4, 5, 6]})
+assert type(df) == datafusion.DataFrame
+# Dataframe:
+# +---+---+
+# | a | b |
+# +---+---+
+# | 1 | 4 |
+# | 2 | 5 |
+# | 3 | 6 |
+# +---+---+
+
+# Create a datafusion DataFrame from a Python list of rows
+df = ctx.from_pylist([{"a": 1, "b": 4}, {"a": 2, "b": 5}, {"a": 3, "b": 6}])
+assert type(df) == datafusion.DataFrame
+
+# Convert pandas DataFrame to datafusion DataFrame
+pandas_df = pd.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})
+df = ctx.from_pandas(pandas_df)
+assert type(df) == datafusion.DataFrame
+
+# Convert polars DataFrame to datafusion DataFrame
+polars_df = pl.DataFrame({"a": [1, 2, 3], "b": [4, 5, 6]})
+df = ctx.from_polars(polars_df)
+assert type(df) == datafusion.DataFrame
+
+# Convert Arrow Table to datafusion DataFrame
+arrow_table = pa.Table.from_pydict({"a": [1, 2, 3], "b": [4, 5, 6]})
+df = ctx.from_arrow_table(arrow_table)
+assert type(df) == datafusion.DataFrame
diff --git a/src/context.rs b/src/context.rs
index 4a19377..4767e47 100644
--- a/src/context.rs
+++ b/src/context.rs
@@ -50,6 +50,7 @@ use datafusion::prelude::{
     AvroReadOptions, CsvReadOptions, DataFrame, NdJsonReadOptions, 
ParquetReadOptions,
 };
 use datafusion_common::ScalarValue;
+use pyo3::types::PyTuple;
 use tokio::runtime::Runtime;
 use tokio::task::JoinHandle;
 
@@ -302,6 +303,80 @@ impl PySessionContext {
         PyDataFrame::new(DataFrame::new(self.ctx.state(), 
plan.plan.as_ref().clone()))
     }
 
+    /// Construct datafusion dataframe from Python list
+    #[allow(clippy::wrong_self_convention)]
+    fn from_pylist(&mut self, data: PyObject, _py: Python) -> 
PyResult<PyDataFrame> {
+        Python::with_gil(|py| {
+            // Instantiate pyarrow Table object & convert to Arrow Table
+            let table_class = py.import("pyarrow")?.getattr("Table")?;
+            let args = PyTuple::new(py, &[data]);
+            let table = table_class.call_method1("from_pylist", args)?.into();
+
+            // Convert Arrow Table to datafusion DataFrame
+            let df = self.from_arrow_table(table, py)?;
+            Ok(df)
+        })
+    }
+
+    /// Construct datafusion dataframe from Python dictionary
+    #[allow(clippy::wrong_self_convention)]
+    fn from_pydict(&mut self, data: PyObject, _py: Python) -> 
PyResult<PyDataFrame> {
+        Python::with_gil(|py| {
+            // Instantiate pyarrow Table object & convert to Arrow Table
+            let table_class = py.import("pyarrow")?.getattr("Table")?;
+            let args = PyTuple::new(py, &[data]);
+            let table = table_class.call_method1("from_pydict", args)?.into();
+
+            // Convert Arrow Table to datafusion DataFrame
+            let df = self.from_arrow_table(table, py)?;
+            Ok(df)
+        })
+    }
+
+    /// Construct datafusion dataframe from Arrow Table
+    #[allow(clippy::wrong_self_convention)]
+    fn from_arrow_table(&mut self, data: PyObject, _py: Python) -> 
PyResult<PyDataFrame> {
+        Python::with_gil(|py| {
+            // Instantiate pyarrow Table object & convert to batches
+            let table = data.call_method0(py, "to_batches")?;
+
+            // Cast PyObject to RecordBatch type
+            // Because create_dataframe() expects a vector of vectors of 
record batches
+            // here we need to wrap the vector of record batches in an 
additional vector
+            let batches = table.extract::<PyArrowType<Vec<RecordBatch>>>(py)?;
+            let list_of_batches = PyArrowType::try_from(vec![batches.0])?;
+            self.create_dataframe(list_of_batches, py)
+        })
+    }
+
+    /// Construct datafusion dataframe from pandas
+    #[allow(clippy::wrong_self_convention)]
+    fn from_pandas(&mut self, data: PyObject, _py: Python) -> 
PyResult<PyDataFrame> {
+        Python::with_gil(|py| {
+            // Instantiate pyarrow Table object & convert to Arrow Table
+            let table_class = py.import("pyarrow")?.getattr("Table")?;
+            let args = PyTuple::new(py, &[data]);
+            let table = table_class.call_method1("from_pandas", args)?.into();
+
+            // Convert Arrow Table to datafusion DataFrame
+            let df = self.from_arrow_table(table, py)?;
+            Ok(df)
+        })
+    }
+
+    /// Construct datafusion dataframe from polars
+    #[allow(clippy::wrong_self_convention)]
+    fn from_polars(&mut self, data: PyObject, _py: Python) -> 
PyResult<PyDataFrame> {
+        Python::with_gil(|py| {
+            // Convert Polars dataframe to Arrow Table
+            let table = data.call_method0(py, "to_arrow")?;
+
+            // Convert Arrow Table to datafusion DataFrame
+            let df = self.from_arrow_table(table, py)?;
+            Ok(df)
+        })
+    }
+
     fn register_table(&mut self, name: &str, table: &PyTable) -> PyResult<()> {
         self.ctx
             .register_table(name, table.table())

Reply via email to