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 2017/04/01 04:00:31 UTC

arrow git commit: ARROW-736: [Python] Mixed-type object DataFrame columns should not silently co…

Repository: arrow
Updated Branches:
  refs/heads/master 067cd4ebf -> d75d7a96c


ARROW-736: [Python] Mixed-type object DataFrame columns should not silently co\u2026

\u2026erce to an Arrow type by default

Author: Phillip Cloud <cp...@gmail.com>

Closes #465 from cpcloud/ARROW-736 and squashes the following commits:

fd09def [Phillip Cloud] Update cmake
bcf6236 [Phillip Cloud] Rename and move
4a18014 [Phillip Cloud] Move test
e80efe1 [Phillip Cloud] Use OwnedRef instead of horror
b2df3e9 [Phillip Cloud] Fix python error handling and make compatible with python27
84d33b4 [Phillip Cloud] ARROW-736: Mixed-type object DataFrame columns should not silently coerce to an Arrow type by default


Project: http://git-wip-us.apache.org/repos/asf/arrow/repo
Commit: http://git-wip-us.apache.org/repos/asf/arrow/commit/d75d7a96
Tree: http://git-wip-us.apache.org/repos/asf/arrow/tree/d75d7a96
Diff: http://git-wip-us.apache.org/repos/asf/arrow/diff/d75d7a96

Branch: refs/heads/master
Commit: d75d7a96ca21bb2c1cfcf3bce8d09c2f24a5b8a6
Parents: 067cd4e
Author: Phillip Cloud <cp...@gmail.com>
Authored: Sat Apr 1 00:00:24 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Sat Apr 1 00:00:24 2017 -0400

----------------------------------------------------------------------
 cpp/src/arrow/python/CMakeLists.txt          |  2 +-
 cpp/src/arrow/python/pandas-test.cc          | 69 ----------------
 cpp/src/arrow/python/pandas_convert.cc       | 52 ++++++++++--
 cpp/src/arrow/python/pandas_convert.h        |  4 +
 cpp/src/arrow/python/python-test.cc          | 96 +++++++++++++++++++++++
 python/pyarrow/tests/test_convert_builtin.py |  5 ++
 python/pyarrow/tests/test_convert_pandas.py  |  5 ++
 7 files changed, 156 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/d75d7a96/cpp/src/arrow/python/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/CMakeLists.txt b/cpp/src/arrow/python/CMakeLists.txt
index 03f5afc..faaad89 100644
--- a/cpp/src/arrow/python/CMakeLists.txt
+++ b/cpp/src/arrow/python/CMakeLists.txt
@@ -88,6 +88,6 @@ install(FILES
 #     INSTALL_RPATH "\$ORIGIN")
 
 if (ARROW_BUILD_TESTS)
-  ADD_ARROW_TEST(pandas-test
+  ADD_ARROW_TEST(python-test
     STATIC_LINK_LIBS "${ARROW_PYTHON_TEST_LINK_LIBS}")
 endif()

http://git-wip-us.apache.org/repos/asf/arrow/blob/d75d7a96/cpp/src/arrow/python/pandas-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/pandas-test.cc b/cpp/src/arrow/python/pandas-test.cc
deleted file mode 100644
index a4e640b..0000000
--- a/cpp/src/arrow/python/pandas-test.cc
+++ /dev/null
@@ -1,69 +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.
-
-#include "gtest/gtest.h"
-
-#include <cstdint>
-#include <memory>
-#include <string>
-#include <vector>
-
-#include "arrow/array.h"
-#include "arrow/builder.h"
-#include "arrow/table.h"
-#include "arrow/test-util.h"
-#include "arrow/type.h"
-
-#include "arrow/python/common.h"
-#include "arrow/python/pandas_convert.h"
-
-namespace arrow {
-namespace py {
-
-TEST(PyBuffer, InvalidInputObject) {
-  PyBuffer buffer(Py_None);
-}
-
-TEST(PandasConversionTest, TestObjectBlockWriteFails) {
-  StringBuilder builder(default_memory_pool());
-  const char value[] = {'\xf1', '\0'};
-
-  for (int i = 0; i < 1000; ++i) {
-    builder.Append(value, static_cast<int32_t>(strlen(value)));
-  }
-
-  std::shared_ptr<Array> arr;
-  ASSERT_OK(builder.Finish(&arr));
-
-  auto f1 = field("f1", utf8());
-  auto f2 = field("f2", utf8());
-  auto f3 = field("f3", utf8());
-  std::vector<std::shared_ptr<Field>> fields = {f1, f2, f3};
-  std::vector<std::shared_ptr<Column>> cols = {std::make_shared<Column>(f1, arr),
-      std::make_shared<Column>(f2, arr), std::make_shared<Column>(f3, arr)};
-
-  auto schema = std::make_shared<Schema>(fields);
-  auto table = std::make_shared<Table>(schema, cols);
-
-  PyObject* out;
-  Py_BEGIN_ALLOW_THREADS;
-  ASSERT_RAISES(UnknownError, ConvertTableToPandas(table, 2, &out));
-  Py_END_ALLOW_THREADS;
-}
-
-}  // namespace py
-}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/d75d7a96/cpp/src/arrow/python/pandas_convert.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/pandas_convert.cc b/cpp/src/arrow/python/pandas_convert.cc
index 68a8d7d..ae9b17c 100644
--- a/cpp/src/arrow/python/pandas_convert.cc
+++ b/cpp/src/arrow/python/pandas_convert.cc
@@ -46,6 +46,7 @@
 #include "arrow/type_fwd.h"
 #include "arrow/type_traits.h"
 #include "arrow/util/bit-util.h"
+#include "arrow/util/logging.h"
 #include "arrow/util/macros.h"
 
 namespace arrow {
@@ -167,8 +168,10 @@ Status AppendObjectStrings(int64_t objects_length, StringBuilder* builder,
       *have_bytes = true;
       const int32_t length = static_cast<int32_t>(PyBytes_GET_SIZE(obj));
       RETURN_NOT_OK(builder->Append(PyBytes_AS_STRING(obj), length));
+    } else if (PyObject_is_null(obj)) {
+      RETURN_NOT_OK(builder->AppendNull());
     } else {
-      builder->AppendNull();
+      return InvalidConversion(obj, "string or bytes");
     }
   }
 
@@ -197,8 +200,10 @@ static Status AppendObjectFixedWidthBytes(int64_t objects_length, int byte_width
       RETURN_NOT_OK(CheckPythonBytesAreFixedLength(obj, byte_width));
       RETURN_NOT_OK(
           builder->Append(reinterpret_cast<const uint8_t*>(PyBytes_AS_STRING(obj))));
+    } else if (PyObject_is_null(obj)) {
+      RETURN_NOT_OK(builder->AppendNull());
     } else {
-      builder->AppendNull();
+      return InvalidConversion(obj, "string or bytes");
     }
   }
 
@@ -413,6 +418,32 @@ inline Status PandasConverter::ConvertData<BooleanType>(std::shared_ptr<Buffer>*
   return Status::OK();
 }
 
+Status InvalidConversion(PyObject* obj, const std::string& expected_type_name) {
+  OwnedRef type(PyObject_Type(obj));
+  RETURN_IF_PYERROR();
+  DCHECK_NE(type.obj(), nullptr);
+
+  OwnedRef type_name(PyObject_GetAttrString(type.obj(), "__name__"));
+  RETURN_IF_PYERROR();
+  DCHECK_NE(type_name.obj(), nullptr);
+
+  OwnedRef bytes_obj(PyUnicode_AsUTF8String(type_name.obj()));
+  RETURN_IF_PYERROR();
+  DCHECK_NE(bytes_obj.obj(), nullptr);
+
+  Py_ssize_t size = PyBytes_GET_SIZE(bytes_obj.obj());
+  const char* bytes = PyBytes_AS_STRING(bytes_obj.obj());
+
+  DCHECK_NE(bytes, nullptr) << "bytes from type(...).__name__ were null";
+
+  std::string cpp_type_name(bytes, size);
+
+  std::stringstream ss;
+  ss << "Python object of type " << cpp_type_name << " is not None and is not a "
+     << expected_type_name << " object";
+  return Status::TypeError(ss.str());
+}
+
 Status PandasConverter::ConvertDates(std::shared_ptr<Array>* out) {
   PyAcquireGIL lock;
 
@@ -427,8 +458,10 @@ Status PandasConverter::ConvertDates(std::shared_ptr<Array>* out) {
     if (PyDate_CheckExact(obj)) {
       PyDateTime_Date* pydate = reinterpret_cast<PyDateTime_Date*>(obj);
       date_builder.Append(PyDate_to_ms(pydate));
-    } else {
+    } else if (PyObject_is_null(obj)) {
       date_builder.AppendNull();
+    } else {
+      return InvalidConversion(obj, "date");
     }
   }
   return date_builder.Finish(out);
@@ -483,14 +516,18 @@ Status PandasConverter::ConvertBooleans(std::shared_ptr<Array>* out) {
   memset(bitmap, 0, nbytes);
 
   int64_t null_count = 0;
+  PyObject* obj;
   for (int64_t i = 0; i < length_; ++i) {
-    if (objects[i] == Py_True) {
+    obj = objects[i];
+    if (obj == Py_True) {
       BitUtil::SetBit(bitmap, i);
       BitUtil::SetBit(null_bitmap_data_, i);
-    } else if (objects[i] != Py_False) {
+    } else if (obj == Py_False) {
+      BitUtil::SetBit(null_bitmap_data_, i);
+    } else if (PyObject_is_null(obj)) {
       ++null_count;
     } else {
-      BitUtil::SetBit(null_bitmap_data_, i);
+      return InvalidConversion(obj, "bool");
     }
   }
 
@@ -551,7 +588,8 @@ Status PandasConverter::ConvertObjects(std::shared_ptr<Array>* out) {
       } else if (PyDate_CheckExact(objects[i])) {
         return ConvertDates(out);
       } else {
-        return Status::TypeError("unhandled python type");
+        return InvalidConversion(
+            const_cast<PyObject*>(objects[i]), "string, bool, or date");
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/arrow/blob/d75d7a96/cpp/src/arrow/python/pandas_convert.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/pandas_convert.h b/cpp/src/arrow/python/pandas_convert.h
index 12644d9..105c159 100644
--- a/cpp/src/arrow/python/pandas_convert.h
+++ b/cpp/src/arrow/python/pandas_convert.h
@@ -24,6 +24,7 @@
 #include <Python.h>
 
 #include <memory>
+#include <string>
 
 #include "arrow/util/visibility.h"
 
@@ -73,6 +74,9 @@ ARROW_EXPORT
 Status PandasObjectsToArrow(MemoryPool* pool, PyObject* ao, PyObject* mo,
     const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out);
 
+ARROW_EXPORT
+Status InvalidConversion(PyObject* obj, const std::string& expected_type_name);
+
 }  // namespace py
 }  // namespace arrow
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/d75d7a96/cpp/src/arrow/python/python-test.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/python-test.cc b/cpp/src/arrow/python/python-test.cc
new file mode 100644
index 0000000..01e30f5
--- /dev/null
+++ b/cpp/src/arrow/python/python-test.cc
@@ -0,0 +1,96 @@
+// 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.
+
+#include "gtest/gtest.h"
+
+#include <memory>
+
+#include <Python.h>
+
+#include "arrow/array.h"
+#include "arrow/builder.h"
+#include "arrow/table.h"
+#include "arrow/test-util.h"
+
+#include "arrow/python/common.h"
+#include "arrow/python/pandas_convert.h"
+#include "arrow/python/builtin_convert.h"
+
+namespace arrow {
+namespace py {
+
+TEST(PyBuffer, InvalidInputObject) {
+  PyBuffer buffer(Py_None);
+}
+
+TEST(PandasConversionTest, TestObjectBlockWriteFails) {
+  StringBuilder builder(default_memory_pool());
+  const char value[] = {'\xf1', '\0'};
+
+  for (int i = 0; i < 1000; ++i) {
+    builder.Append(value, static_cast<int32_t>(strlen(value)));
+  }
+
+  std::shared_ptr<Array> arr;
+  ASSERT_OK(builder.Finish(&arr));
+
+  auto f1 = field("f1", utf8());
+  auto f2 = field("f2", utf8());
+  auto f3 = field("f3", utf8());
+  std::vector<std::shared_ptr<Field>> fields = {f1, f2, f3};
+  std::vector<std::shared_ptr<Column>> cols = {std::make_shared<Column>(f1, arr),
+      std::make_shared<Column>(f2, arr), std::make_shared<Column>(f3, arr)};
+
+  auto schema = std::make_shared<Schema>(fields);
+  auto table = std::make_shared<Table>(schema, cols);
+
+  PyObject* out;
+  Py_BEGIN_ALLOW_THREADS;
+  ASSERT_RAISES(UnknownError, ConvertTableToPandas(table, 2, &out));
+  Py_END_ALLOW_THREADS;
+}
+
+TEST(BuiltinConversionTest, TestMixedTypeFails) {
+  PyAcquireGIL lock;
+  MemoryPool* pool = default_memory_pool();
+  std::shared_ptr<Array> arr;
+
+  OwnedRef list_ref(PyList_New(3));
+  PyObject* list = list_ref.obj();
+
+  ASSERT_NE(list, nullptr);
+
+  PyObject* str = PyUnicode_FromString("abc");
+  ASSERT_NE(str, nullptr);
+
+  PyObject* integer = PyLong_FromLong(1234L);
+  ASSERT_NE(integer, nullptr);
+
+  PyObject* doub = PyFloat_FromDouble(123.0234);
+  ASSERT_NE(doub, nullptr);
+
+  // This steals a reference to each object, so we don't need to decref them later
+  // just the list
+  ASSERT_EQ(PyList_SetItem(list, 0, str), 0);
+  ASSERT_EQ(PyList_SetItem(list, 1, integer), 0);
+  ASSERT_EQ(PyList_SetItem(list, 2, doub), 0);
+
+  ASSERT_RAISES(UnknownError, ConvertPySequence(list, pool, &arr));
+}
+
+}  // namespace py
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/d75d7a96/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 9925125..3309ba0 100644
--- a/python/pyarrow/tests/test_convert_builtin.py
+++ b/python/pyarrow/tests/test_convert_builtin.py
@@ -157,3 +157,8 @@ class TestConvertList(unittest.TestCase):
         assert arr.null_count == 1
         assert arr.type == pyarrow.list_(pyarrow.int64())
         assert arr.to_pylist() == data
+
+    def test_mixed_types_fails(self):
+        data = ['a', 1, 2.0]
+        with self.assertRaises(pyarrow.error.ArrowException):
+            pyarrow.from_pylist(data)

http://git-wip-us.apache.org/repos/asf/arrow/blob/d75d7a96/python/pyarrow/tests/test_convert_pandas.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_convert_pandas.py b/python/pyarrow/tests/test_convert_pandas.py
index f7cb47f..3f19b68 100644
--- a/python/pyarrow/tests/test_convert_pandas.py
+++ b/python/pyarrow/tests/test_convert_pandas.py
@@ -398,3 +398,8 @@ class TestPandasConversion(unittest.TestCase):
         ]
         for values in arrays:
             self._check_array_roundtrip(values)
+
+    def test_mixed_types_fails(self):
+        data = pd.DataFrame({'a': ['a', 1, 2.0]})
+        with self.assertRaises(A.error.ArrowException):
+            A.Table.from_pandas(data)