You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2016/11/07 18:52:43 UTC
arrow git commit: ARROW-362: Fix memory leak in zero-copy arrow to
NumPy/pandas conversion
Repository: arrow
Updated Branches:
refs/heads/master 121e82682 -> 79344b335
ARROW-362: Fix memory leak in zero-copy arrow to NumPy/pandas conversion
close #198
Author: Wes McKinney <we...@twosigma.com>
Author: Uwe L. Korn <uw...@xhochy.com>
Closes #200 from wesm/ARROW-362 and squashes the following commits:
99df96b [Wes McKinney] Force gc to avoid non-deterministic failure
d85228f [Wes McKinney] Be more careful about reference counts in zero-copy handoff, add pyarrow.Array.to_pandas method
cc7a6b3 [Uwe L. Korn] ARROW-362: Remove redunant reference count
Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/79344b33
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/79344b33
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/79344b33
Branch: refs/heads/master
Commit: 79344b335849c2eb43954b0751018051814019d6
Parents: 121e826
Author: Wes McKinney <we...@twosigma.com>
Authored: Mon Nov 7 13:52:32 2016 -0500
Committer: Wes McKinney <we...@twosigma.com>
Committed: Mon Nov 7 13:52:32 2016 -0500
----------------------------------------------------------------------
python/pyarrow/array.pyx | 21 ++++++++++++++++
python/pyarrow/includes/common.pxd | 7 ++++++
python/pyarrow/includes/pyarrow.pxd | 4 ++--
python/pyarrow/table.pyx | 18 +++++++++-----
python/pyarrow/tests/test_array.py | 29 +++++++++++++++++++++++
python/pyarrow/tests/test_convert_builtin.py | 4 ++++
python/src/pyarrow/adapters/pandas.cc | 4 +---
7 files changed, 76 insertions(+), 11 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/arrow/blob/79344b33/python/pyarrow/array.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/array.pyx b/python/pyarrow/array.pyx
index 84ab4a4..fbe4e38 100644
--- a/python/pyarrow/array.pyx
+++ b/python/pyarrow/array.pyx
@@ -22,6 +22,7 @@
import numpy as np
from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.common cimport PyObject_to_object
cimport pyarrow.includes.pyarrow as pyarrow
import pyarrow.config
@@ -35,6 +36,8 @@ from pyarrow.scalar import NA
from pyarrow.schema cimport Schema
import pyarrow.schema as schema
+cimport cpython
+
def total_allocated_bytes():
cdef MemoryPool* pool = pyarrow.get_memory_pool()
@@ -111,6 +114,24 @@ cdef class Array:
def slice(self, start, end):
pass
+ def to_pandas(self):
+ """
+ Convert to an array object suitable for use in pandas
+
+ See also
+ --------
+ Column.to_pandas
+ Table.to_pandas
+ RecordBatch.to_pandas
+ """
+ cdef:
+ PyObject* np_arr
+
+ check_status(pyarrow.ConvertArrayToPandas(
+ self.sp_array, <PyObject*> self, &np_arr))
+
+ return PyObject_to_object(np_arr)
+
cdef class NullArray(Array):
pass
http://git-wip-us.apache.org/repos/asf/arrow/blob/79344b33/python/pyarrow/includes/common.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/common.pxd b/python/pyarrow/includes/common.pxd
index 05c0123..f689bdc 100644
--- a/python/pyarrow/includes/common.pxd
+++ b/python/pyarrow/includes/common.pxd
@@ -47,3 +47,10 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
c_bool IsKeyError()
c_bool IsNotImplemented()
c_bool IsInvalid()
+
+
+cdef inline object PyObject_to_object(PyObject* o):
+ # Cast to "object" increments reference count
+ cdef object result = <object> o
+ cpython.Py_DECREF(result)
+ return result
http://git-wip-us.apache.org/repos/asf/arrow/blob/79344b33/python/pyarrow/includes/pyarrow.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/pyarrow.pxd b/python/pyarrow/includes/pyarrow.pxd
index e1da191..a5444c2 100644
--- a/python/pyarrow/includes/pyarrow.pxd
+++ b/python/pyarrow/includes/pyarrow.pxd
@@ -34,10 +34,10 @@ cdef extern from "pyarrow/api.h" namespace "pyarrow" nogil:
shared_ptr[CArray]* out)
CStatus ConvertArrayToPandas(const shared_ptr[CArray]& arr,
- object py_ref, PyObject** out)
+ PyObject* py_ref, PyObject** out)
CStatus ConvertColumnToPandas(const shared_ptr[CColumn]& arr,
- object py_ref, PyObject** out)
+ PyObject* py_ref, PyObject** out)
MemoryPool* get_memory_pool()
http://git-wip-us.apache.org/repos/asf/arrow/blob/79344b33/python/pyarrow/table.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/table.pyx b/python/pyarrow/table.pyx
index 9695712..c71bc71 100644
--- a/python/pyarrow/table.pyx
+++ b/python/pyarrow/table.pyx
@@ -22,6 +22,7 @@
from cython.operator cimport dereference as deref
from pyarrow.includes.libarrow cimport *
+from pyarrow.includes.common cimport PyObject_to_object
cimport pyarrow.includes.pyarrow as pyarrow
import pyarrow.config
@@ -32,6 +33,7 @@ from pyarrow.schema cimport box_data_type, box_schema
from pyarrow.compat import frombytes, tobytes
+cimport cpython
cdef class ChunkedArray:
'''
@@ -100,8 +102,10 @@ cdef class Column:
import pandas as pd
- check_status(pyarrow.ConvertColumnToPandas(self.sp_column, self, &arr))
- return pd.Series(<object>arr, name=self.name)
+ check_status(pyarrow.ConvertColumnToPandas(self.sp_column,
+ <PyObject*> self, &arr))
+
+ return pd.Series(PyObject_to_object(arr), name=self.name)
cdef _check_nullptr(self):
if self.column == NULL:
@@ -248,9 +252,10 @@ cdef class RecordBatch:
data = []
for i in range(self.batch.num_columns()):
arr = self.batch.column(i)
- check_status(pyarrow.ConvertArrayToPandas(arr, self, &np_arr))
+ check_status(pyarrow.ConvertArrayToPandas(arr, <PyObject*> self,
+ &np_arr))
names.append(frombytes(self.batch.column_name(i)))
- data.append(<object> np_arr)
+ data.append(PyObject_to_object(np_arr))
return pd.DataFrame(dict(zip(names, data)), columns=names)
@@ -375,9 +380,10 @@ cdef class Table:
for i in range(self.table.num_columns()):
col = self.table.column(i)
column = self.column(i)
- check_status(pyarrow.ConvertColumnToPandas(col, column, &arr))
+ check_status(pyarrow.ConvertColumnToPandas(
+ col, <PyObject*> column, &arr))
names.append(frombytes(col.get().name()))
- data.append(<object> arr)
+ data.append(PyObject_to_object(arr))
return pd.DataFrame(dict(zip(names, data)), columns=names)
http://git-wip-us.apache.org/repos/asf/arrow/blob/79344b33/python/pyarrow/tests/test_array.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_array.py b/python/pyarrow/tests/test_array.py
index 0a17f69..ead17db 100644
--- a/python/pyarrow/tests/test_array.py
+++ b/python/pyarrow/tests/test_array.py
@@ -15,6 +15,8 @@
# specific language governing permissions and limitations
# under the License.
+import sys
+
import pyarrow
import pyarrow.formatting as fmt
@@ -71,3 +73,30 @@ def test_long_array_format():
99
]"""
assert result == expected
+
+
+def test_to_pandas_zero_copy():
+ import gc
+
+ arr = pyarrow.from_pylist(range(10))
+
+ for i in range(10):
+ np_arr = arr.to_pandas()
+ assert sys.getrefcount(np_arr) == 2
+ np_arr = None # noqa
+
+ assert sys.getrefcount(arr) == 2
+
+ for i in range(10):
+ arr = pyarrow.from_pylist(range(10))
+ np_arr = arr.to_pandas()
+ arr = None
+ gc.collect()
+
+ # Ensure base is still valid
+
+ # Because of py.test's assert inspection magic, if you put getrefcount
+ # on the line being examined, it will be 1 higher than you expect
+ base_refcount = sys.getrefcount(np_arr.base)
+ assert base_refcount == 2
+ np_arr.sum()
http://git-wip-us.apache.org/repos/asf/arrow/blob/79344b33/python/pyarrow/tests/test_convert_builtin.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_convert_builtin.py b/python/pyarrow/tests/test_convert_builtin.py
index 2beb6b3..8937f8d 100644
--- a/python/pyarrow/tests/test_convert_builtin.py
+++ b/python/pyarrow/tests/test_convert_builtin.py
@@ -47,6 +47,10 @@ class TestConvertList(unittest.TestCase):
def test_garbage_collection(self):
import gc
+
+ # Force the cyclic garbage collector to run
+ gc.collect()
+
bytes_before = pyarrow.total_allocated_bytes()
pyarrow.from_pylist([1, None, 3, None])
gc.collect()
http://git-wip-us.apache.org/repos/asf/arrow/blob/79344b33/python/src/pyarrow/adapters/pandas.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/pandas.cc b/python/src/pyarrow/adapters/pandas.cc
index 7e70be7..6a3966b 100644
--- a/python/src/pyarrow/adapters/pandas.cc
+++ b/python/src/pyarrow/adapters/pandas.cc
@@ -628,8 +628,6 @@ class ArrowDeserializer {
PyAcquireGIL lock;
// Zero-Copy. We can pass the data pointer directly to NumPy.
- Py_INCREF(py_ref_);
- OwnedRef py_ref(py_ref_);
npy_intp dims[1] = {col_->length()};
out_ = reinterpret_cast<PyArrayObject*>(PyArray_SimpleNewFromData(1, dims,
type, data));
@@ -646,7 +644,7 @@ class ArrowDeserializer {
return Status::OK();
} else {
// PyArray_SetBaseObject steals our reference to py_ref_
- py_ref.release();
+ Py_INCREF(py_ref_);
}
// Arrow data is immutable.