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/03/26 15:45:43 UTC

[1/5] arrow git commit: ARROW-341: [Python] Move pyarrow's C++ code to the main C++ source tree, install libarrow_python and headers

Repository: arrow
Updated Branches:
  refs/heads/master 6d4e86290 -> 3aac4adef


http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/io.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/io.cc b/python/src/pyarrow/io.cc
deleted file mode 100644
index c66155b..0000000
--- a/python/src/pyarrow/io.cc
+++ /dev/null
@@ -1,221 +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 "pyarrow/io.h"
-
-#include <cstdint>
-#include <cstdlib>
-
-#include "arrow/io/memory.h"
-#include "arrow/memory_pool.h"
-#include "arrow/status.h"
-
-#include "pyarrow/common.h"
-
-namespace arrow {
-namespace py {
-
-// ----------------------------------------------------------------------
-// Python file
-
-PythonFile::PythonFile(PyObject* file) : file_(file) {
-  Py_INCREF(file_);
-}
-
-PythonFile::~PythonFile() {
-  Py_DECREF(file_);
-}
-
-static Status CheckPyError() {
-  if (PyErr_Occurred()) {
-    PyObject *exc_type, *exc_value, *traceback;
-    PyErr_Fetch(&exc_type, &exc_value, &traceback);
-    PyObjectStringify stringified(exc_value);
-    std::string message(stringified.bytes);
-    Py_XDECREF(exc_type);
-    Py_XDECREF(exc_value);
-    Py_XDECREF(traceback);
-    PyErr_Clear();
-    return Status::IOError(message);
-  }
-  return Status::OK();
-}
-
-// This is annoying: because C++11 does not allow implicit conversion of string
-// literals to non-const char*, we need to go through some gymnastics to use
-// PyObject_CallMethod without a lot of pain (its arguments are non-const
-// char*)
-template <typename... ArgTypes>
-static inline PyObject* cpp_PyObject_CallMethod(
-    PyObject* obj, const char* method_name, const char* argspec, ArgTypes... args) {
-  return PyObject_CallMethod(
-      obj, const_cast<char*>(method_name), const_cast<char*>(argspec), args...);
-}
-
-Status PythonFile::Close() {
-  // whence: 0 for relative to start of file, 2 for end of file
-  PyObject* result = cpp_PyObject_CallMethod(file_, "close", "()");
-  Py_XDECREF(result);
-  ARROW_RETURN_NOT_OK(CheckPyError());
-  return Status::OK();
-}
-
-Status PythonFile::Seek(int64_t position, int whence) {
-  // whence: 0 for relative to start of file, 2 for end of file
-  PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ii)", position, whence);
-  Py_XDECREF(result);
-  ARROW_RETURN_NOT_OK(CheckPyError());
-  return Status::OK();
-}
-
-Status PythonFile::Read(int64_t nbytes, PyObject** out) {
-  PyObject* result = cpp_PyObject_CallMethod(file_, "read", "(i)", nbytes);
-  ARROW_RETURN_NOT_OK(CheckPyError());
-  *out = result;
-  return Status::OK();
-}
-
-Status PythonFile::Write(const uint8_t* data, int64_t nbytes) {
-  PyObject* py_data =
-      PyBytes_FromStringAndSize(reinterpret_cast<const char*>(data), nbytes);
-  ARROW_RETURN_NOT_OK(CheckPyError());
-
-  PyObject* result = cpp_PyObject_CallMethod(file_, "write", "(O)", py_data);
-  Py_XDECREF(py_data);
-  Py_XDECREF(result);
-  ARROW_RETURN_NOT_OK(CheckPyError());
-  return Status::OK();
-}
-
-Status PythonFile::Tell(int64_t* position) {
-  PyObject* result = cpp_PyObject_CallMethod(file_, "tell", "()");
-  ARROW_RETURN_NOT_OK(CheckPyError());
-
-  *position = PyLong_AsLongLong(result);
-  Py_DECREF(result);
-
-  // PyLong_AsLongLong can raise OverflowError
-  ARROW_RETURN_NOT_OK(CheckPyError());
-
-  return Status::OK();
-}
-
-// ----------------------------------------------------------------------
-// Seekable input stream
-
-PyReadableFile::PyReadableFile(PyObject* file) {
-  file_.reset(new PythonFile(file));
-}
-
-PyReadableFile::~PyReadableFile() {}
-
-Status PyReadableFile::Close() {
-  PyAcquireGIL lock;
-  return file_->Close();
-}
-
-Status PyReadableFile::Seek(int64_t position) {
-  PyAcquireGIL lock;
-  return file_->Seek(position, 0);
-}
-
-Status PyReadableFile::Tell(int64_t* position) {
-  PyAcquireGIL lock;
-  return file_->Tell(position);
-}
-
-Status PyReadableFile::Read(int64_t nbytes, int64_t* bytes_read, uint8_t* out) {
-  PyAcquireGIL lock;
-  PyObject* bytes_obj;
-  ARROW_RETURN_NOT_OK(file_->Read(nbytes, &bytes_obj));
-
-  *bytes_read = PyBytes_GET_SIZE(bytes_obj);
-  std::memcpy(out, PyBytes_AS_STRING(bytes_obj), *bytes_read);
-  Py_DECREF(bytes_obj);
-
-  return Status::OK();
-}
-
-Status PyReadableFile::Read(int64_t nbytes, std::shared_ptr<Buffer>* out) {
-  PyAcquireGIL lock;
-
-  PyObject* bytes_obj;
-  ARROW_RETURN_NOT_OK(file_->Read(nbytes, &bytes_obj));
-
-  *out = std::make_shared<PyBuffer>(bytes_obj);
-  Py_DECREF(bytes_obj);
-
-  return Status::OK();
-}
-
-Status PyReadableFile::GetSize(int64_t* size) {
-  PyAcquireGIL lock;
-
-  int64_t current_position;
-  ;
-  ARROW_RETURN_NOT_OK(file_->Tell(&current_position));
-
-  ARROW_RETURN_NOT_OK(file_->Seek(0, 2));
-
-  int64_t file_size;
-  ARROW_RETURN_NOT_OK(file_->Tell(&file_size));
-
-  // Restore previous file position
-  ARROW_RETURN_NOT_OK(file_->Seek(current_position, 0));
-
-  *size = file_size;
-  return Status::OK();
-}
-
-bool PyReadableFile::supports_zero_copy() const {
-  return false;
-}
-
-// ----------------------------------------------------------------------
-// Output stream
-
-PyOutputStream::PyOutputStream(PyObject* file) {
-  file_.reset(new PythonFile(file));
-}
-
-PyOutputStream::~PyOutputStream() {}
-
-Status PyOutputStream::Close() {
-  PyAcquireGIL lock;
-  return file_->Close();
-}
-
-Status PyOutputStream::Tell(int64_t* position) {
-  PyAcquireGIL lock;
-  return file_->Tell(position);
-}
-
-Status PyOutputStream::Write(const uint8_t* data, int64_t nbytes) {
-  PyAcquireGIL lock;
-  return file_->Write(data, nbytes);
-}
-
-// ----------------------------------------------------------------------
-// A readable file that is backed by a PyBuffer
-
-PyBytesReader::PyBytesReader(PyObject* obj)
-    : io::BufferReader(std::make_shared<PyBuffer>(obj)) {}
-
-PyBytesReader::~PyBytesReader() {}
-
-}  // namespace py
-}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/io.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/io.h b/python/src/pyarrow/io.h
deleted file mode 100644
index 89af609..0000000
--- a/python/src/pyarrow/io.h
+++ /dev/null
@@ -1,99 +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.
-
-#ifndef PYARROW_IO_H
-#define PYARROW_IO_H
-
-#include "arrow/io/interfaces.h"
-#include "arrow/io/memory.h"
-#include "arrow/util/visibility.h"
-
-#include "pyarrow/config.h"
-
-#include "pyarrow/common.h"
-
-namespace arrow {
-
-class MemoryPool;
-
-namespace py {
-
-// A common interface to a Python file-like object. Must acquire GIL before
-// calling any methods
-class PythonFile {
- public:
-  PythonFile(PyObject* file);
-  ~PythonFile();
-
-  Status Close();
-  Status Seek(int64_t position, int whence);
-  Status Read(int64_t nbytes, PyObject** out);
-  Status Tell(int64_t* position);
-  Status Write(const uint8_t* data, int64_t nbytes);
-
- private:
-  PyObject* file_;
-};
-
-class ARROW_EXPORT PyReadableFile : public io::RandomAccessFile {
- public:
-  explicit PyReadableFile(PyObject* file);
-  virtual ~PyReadableFile();
-
-  Status Close() override;
-
-  Status Read(int64_t nbytes, int64_t* bytes_read, uint8_t* out) override;
-  Status Read(int64_t nbytes, std::shared_ptr<Buffer>* out) override;
-
-  Status GetSize(int64_t* size) override;
-
-  Status Seek(int64_t position) override;
-
-  Status Tell(int64_t* position) override;
-
-  bool supports_zero_copy() const override;
-
- private:
-  std::unique_ptr<PythonFile> file_;
-};
-
-class ARROW_EXPORT PyOutputStream : public io::OutputStream {
- public:
-  explicit PyOutputStream(PyObject* file);
-  virtual ~PyOutputStream();
-
-  Status Close() override;
-  Status Tell(int64_t* position) override;
-  Status Write(const uint8_t* data, int64_t nbytes) override;
-
- private:
-  std::unique_ptr<PythonFile> file_;
-};
-
-// A zero-copy reader backed by a PyBuffer object
-class ARROW_EXPORT PyBytesReader : public io::BufferReader {
- public:
-  explicit PyBytesReader(PyObject* obj);
-  virtual ~PyBytesReader();
-};
-
-// TODO(wesm): seekable output files
-
-}  // namespace py
-}  // namespace arrow
-
-#endif  // PYARROW_IO_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/numpy_interop.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/numpy_interop.h b/python/src/pyarrow/numpy_interop.h
deleted file mode 100644
index 57f3328..0000000
--- a/python/src/pyarrow/numpy_interop.h
+++ /dev/null
@@ -1,60 +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.
-
-#ifndef PYARROW_NUMPY_INTEROP_H
-#define PYARROW_NUMPY_INTEROP_H
-
-#include <Python.h>
-
-#include <numpy/numpyconfig.h>
-
-// Don't use the deprecated Numpy functions
-#ifdef NPY_1_7_API_VERSION
-#define NPY_NO_DEPRECATED_API NPY_1_7_API_VERSION
-#else
-#define NPY_ARRAY_NOTSWAPPED NPY_NOTSWAPPED
-#define NPY_ARRAY_ALIGNED NPY_ALIGNED
-#define NPY_ARRAY_WRITEABLE NPY_WRITEABLE
-#define NPY_ARRAY_UPDATEIFCOPY NPY_UPDATEIFCOPY
-#endif
-
-// This is required to be able to access the NumPy C API properly in C++ files
-// other than this main one
-#define PY_ARRAY_UNIQUE_SYMBOL pyarrow_ARRAY_API
-#ifndef NUMPY_IMPORT_ARRAY
-#define NO_IMPORT_ARRAY
-#endif
-
-#include <numpy/arrayobject.h>
-#include <numpy/ufuncobject.h>
-
-namespace arrow {
-namespace py {
-
-inline int import_numpy() {
-#ifdef NUMPY_IMPORT_ARRAY
-  import_array1(-1);
-  import_umath1(-1);
-#endif
-
-  return 0;
-}
-
-}  // namespace py
-}  // namespace arrow
-
-#endif  // PYARROW_NUMPY_INTEROP_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/type_traits.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/type_traits.h b/python/src/pyarrow/type_traits.h
deleted file mode 100644
index cc65d5c..0000000
--- a/python/src/pyarrow/type_traits.h
+++ /dev/null
@@ -1,212 +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 <Python.h>
-
-#include <cstdint>
-
-#include "pyarrow/numpy_interop.h"
-
-#include "arrow/builder.h"
-#include "arrow/type.h"
-
-namespace arrow {
-namespace py {
-
-template <int TYPE>
-struct npy_traits {};
-
-template <>
-struct npy_traits<NPY_BOOL> {
-  typedef uint8_t value_type;
-  using TypeClass = BooleanType;
-  using BuilderClass = BooleanBuilder;
-
-  static constexpr bool supports_nulls = false;
-  static inline bool isnull(uint8_t v) { return false; }
-};
-
-#define NPY_INT_DECL(TYPE, CapType, T)               \
-  template <>                                        \
-  struct npy_traits<NPY_##TYPE> {                    \
-    typedef T value_type;                            \
-    using TypeClass = CapType##Type;                 \
-    using BuilderClass = CapType##Builder;           \
-                                                     \
-    static constexpr bool supports_nulls = false;    \
-    static inline bool isnull(T v) { return false; } \
-  };
-
-NPY_INT_DECL(INT8, Int8, int8_t);
-NPY_INT_DECL(INT16, Int16, int16_t);
-NPY_INT_DECL(INT32, Int32, int32_t);
-NPY_INT_DECL(INT64, Int64, int64_t);
-
-NPY_INT_DECL(UINT8, UInt8, uint8_t);
-NPY_INT_DECL(UINT16, UInt16, uint16_t);
-NPY_INT_DECL(UINT32, UInt32, uint32_t);
-NPY_INT_DECL(UINT64, UInt64, uint64_t);
-
-#if NPY_INT64 != NPY_LONGLONG
-NPY_INT_DECL(LONGLONG, Int64, int64_t);
-NPY_INT_DECL(ULONGLONG, UInt64, uint64_t);
-#endif
-
-template <>
-struct npy_traits<NPY_FLOAT32> {
-  typedef float value_type;
-  using TypeClass = FloatType;
-  using BuilderClass = FloatBuilder;
-
-  static constexpr bool supports_nulls = true;
-
-  static inline bool isnull(float v) { return v != v; }
-};
-
-template <>
-struct npy_traits<NPY_FLOAT64> {
-  typedef double value_type;
-  using TypeClass = DoubleType;
-  using BuilderClass = DoubleBuilder;
-
-  static constexpr bool supports_nulls = true;
-
-  static inline bool isnull(double v) { return v != v; }
-};
-
-template <>
-struct npy_traits<NPY_DATETIME> {
-  typedef int64_t value_type;
-  using TypeClass = TimestampType;
-  using BuilderClass = TimestampBuilder;
-
-  static constexpr bool supports_nulls = true;
-
-  static inline bool isnull(int64_t v) {
-    // NaT = -2**63
-    // = -0x8000000000000000
-    // = -9223372036854775808;
-    // = std::numeric_limits<int64_t>::min()
-    return v == std::numeric_limits<int64_t>::min();
-  }
-};
-
-template <>
-struct npy_traits<NPY_OBJECT> {
-  typedef PyObject* value_type;
-  static constexpr bool supports_nulls = true;
-};
-
-template <int TYPE>
-struct arrow_traits {};
-
-template <>
-struct arrow_traits<Type::BOOL> {
-  static constexpr int npy_type = NPY_BOOL;
-  static constexpr bool supports_nulls = false;
-  static constexpr bool is_boolean = true;
-  static constexpr bool is_numeric_not_nullable = false;
-  static constexpr bool is_numeric_nullable = false;
-};
-
-#define INT_DECL(TYPE)                                     \
-  template <>                                              \
-  struct arrow_traits<Type::TYPE> {                        \
-    static constexpr int npy_type = NPY_##TYPE;            \
-    static constexpr bool supports_nulls = false;          \
-    static constexpr double na_value = NAN;                \
-    static constexpr bool is_boolean = false;              \
-    static constexpr bool is_numeric_not_nullable = true;  \
-    static constexpr bool is_numeric_nullable = false;     \
-    typedef typename npy_traits<NPY_##TYPE>::value_type T; \
-  };
-
-INT_DECL(INT8);
-INT_DECL(INT16);
-INT_DECL(INT32);
-INT_DECL(INT64);
-INT_DECL(UINT8);
-INT_DECL(UINT16);
-INT_DECL(UINT32);
-INT_DECL(UINT64);
-
-template <>
-struct arrow_traits<Type::FLOAT> {
-  static constexpr int npy_type = NPY_FLOAT32;
-  static constexpr bool supports_nulls = true;
-  static constexpr float na_value = NAN;
-  static constexpr bool is_boolean = false;
-  static constexpr bool is_numeric_not_nullable = false;
-  static constexpr bool is_numeric_nullable = true;
-  typedef typename npy_traits<NPY_FLOAT32>::value_type T;
-};
-
-template <>
-struct arrow_traits<Type::DOUBLE> {
-  static constexpr int npy_type = NPY_FLOAT64;
-  static constexpr bool supports_nulls = true;
-  static constexpr double na_value = NAN;
-  static constexpr bool is_boolean = false;
-  static constexpr bool is_numeric_not_nullable = false;
-  static constexpr bool is_numeric_nullable = true;
-  typedef typename npy_traits<NPY_FLOAT64>::value_type T;
-};
-
-static constexpr int64_t kPandasTimestampNull = std::numeric_limits<int64_t>::min();
-
-template <>
-struct arrow_traits<Type::TIMESTAMP> {
-  static constexpr int npy_type = NPY_DATETIME;
-  static constexpr bool supports_nulls = true;
-  static constexpr int64_t na_value = kPandasTimestampNull;
-  static constexpr bool is_boolean = false;
-  static constexpr bool is_numeric_not_nullable = false;
-  static constexpr bool is_numeric_nullable = true;
-  typedef typename npy_traits<NPY_DATETIME>::value_type T;
-};
-
-template <>
-struct arrow_traits<Type::DATE64> {
-  static constexpr int npy_type = NPY_DATETIME;
-  static constexpr bool supports_nulls = true;
-  static constexpr int64_t na_value = kPandasTimestampNull;
-  static constexpr bool is_boolean = false;
-  static constexpr bool is_numeric_not_nullable = false;
-  static constexpr bool is_numeric_nullable = true;
-  typedef typename npy_traits<NPY_DATETIME>::value_type T;
-};
-
-template <>
-struct arrow_traits<Type::STRING> {
-  static constexpr int npy_type = NPY_OBJECT;
-  static constexpr bool supports_nulls = true;
-  static constexpr bool is_boolean = false;
-  static constexpr bool is_numeric_not_nullable = false;
-  static constexpr bool is_numeric_nullable = false;
-};
-
-template <>
-struct arrow_traits<Type::BINARY> {
-  static constexpr int npy_type = NPY_OBJECT;
-  static constexpr bool supports_nulls = true;
-  static constexpr bool is_boolean = false;
-  static constexpr bool is_numeric_not_nullable = false;
-  static constexpr bool is_numeric_nullable = false;
-};
-
-}  // namespace py
-}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/util/CMakeLists.txt b/python/src/pyarrow/util/CMakeLists.txt
deleted file mode 100644
index 6cd49cb..0000000
--- a/python/src/pyarrow/util/CMakeLists.txt
+++ /dev/null
@@ -1,39 +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.
-
-#######################################
-# pyarrow_test_main
-#######################################
-
-if (PYARROW_BUILD_TESTS)
-  add_library(pyarrow_test_main STATIC
-	test_main.cc)
-
-  if (APPLE)
-	target_link_libraries(pyarrow_test_main
-      gtest
-      dl)
-	set_target_properties(pyarrow_test_main
-      PROPERTIES LINK_FLAGS "-undefined dynamic_lookup")
-  else()
-	target_link_libraries(pyarrow_test_main
-      gtest
-      pthread
-      dl
-	  )
-  endif()
-endif()

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/util/datetime.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/util/datetime.h b/python/src/pyarrow/util/datetime.h
deleted file mode 100644
index f704a96..0000000
--- a/python/src/pyarrow/util/datetime.h
+++ /dev/null
@@ -1,42 +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.
-
-#ifndef PYARROW_UTIL_DATETIME_H
-#define PYARROW_UTIL_DATETIME_H
-
-#include <Python.h>
-#include <datetime.h>
-
-namespace arrow {
-namespace py {
-
-inline int64_t PyDate_to_ms(PyDateTime_Date* pydate) {
-  struct tm date = {0};
-  date.tm_year = PyDateTime_GET_YEAR(pydate) - 1900;
-  date.tm_mon = PyDateTime_GET_MONTH(pydate) - 1;
-  date.tm_mday = PyDateTime_GET_DAY(pydate);
-  struct tm epoch = {0};
-  epoch.tm_year = 70;
-  epoch.tm_mday = 1;
-  // Milliseconds since the epoch
-  return lrint(difftime(mktime(&date), mktime(&epoch)) * 1000);
-}
-
-}  // namespace py
-}  // namespace arrow
-
-#endif  // PYARROW_UTIL_DATETIME_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/util/test_main.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/util/test_main.cc b/python/src/pyarrow/util/test_main.cc
deleted file mode 100644
index d8d1d03..0000000
--- a/python/src/pyarrow/util/test_main.cc
+++ /dev/null
@@ -1,36 +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 <Python.h>
-
-#include <gtest/gtest.h>
-
-#include "pyarrow/do_import_numpy.h"
-#include "pyarrow/numpy_interop.h"
-
-int main(int argc, char** argv) {
-  ::testing::InitGoogleTest(&argc, argv);
-
-  Py_Initialize();
-  arrow::py::import_numpy();
-
-  int ret = RUN_ALL_TESTS();
-
-  Py_Finalize();
-
-  return ret;
-}


[3/5] arrow git commit: ARROW-341: [Python] Move pyarrow's C++ code to the main C++ source tree, install libarrow_python and headers

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/python/CMakeLists.txt b/python/CMakeLists.txt
index ef874e3..35a1a89 100644
--- a/python/CMakeLists.txt
+++ b/python/CMakeLists.txt
@@ -47,9 +47,6 @@ endif()
 
 # Top level cmake dir
 if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}")
-  option(PYARROW_BUILD_TESTS
-    "Build the PyArrow C++ googletest unit tests"
-    OFF)
   option(PYARROW_BUILD_PARQUET
     "Build the PyArrow Parquet integration"
     OFF)
@@ -57,7 +54,7 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}")
     "Build the PyArrow jemalloc integration"
     OFF)
   option(PYARROW_BUNDLE_ARROW_CPP
-    "Bundle the Arrow C++ libraries" 
+    "Bundle the Arrow C++ libraries"
     OFF)
 endif()
 
@@ -75,6 +72,8 @@ endif(CCACHE_FOUND)
 # Compiler flags
 ############################################################
 
+include(BuildUtils)
+include(CompilerInfo)
 include(SetupCxxFlags)
 
 # Add common flags
@@ -86,8 +85,6 @@ set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fno-omit-frame-pointer")
 # Suppress Cython warnings
 set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wno-unused-variable")
 
-# Determine compiler version
-include(CompilerInfo)
 
 if ("${COMPILER_FAMILY}" STREQUAL "clang")
   # Using Clang with ccache causes a bunch of spurious warnings that are
@@ -216,115 +213,8 @@ include_directories(SYSTEM
   src)
 
 ############################################################
-# Testing
-############################################################
-
-# Add a new test case, with or without an executable that should be built.
-#
-# REL_TEST_NAME is the name of the test. It may be a single component
-# (e.g. monotime-test) or contain additional components (e.g.
-# net/net_util-test). Either way, the last component must be a globally
-# unique name.
-#
-# Arguments after the test name will be passed to set_tests_properties().
-function(ADD_PYARROW_TEST REL_TEST_NAME)
-  if(NO_TESTS)
-    return()
-  endif()
-  get_filename_component(TEST_NAME ${REL_TEST_NAME} NAME_WE)
-
-  if(EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/${REL_TEST_NAME}.cc)
-    # This test has a corresponding .cc file, set it up as an executable.
-    set(TEST_PATH "${EXECUTABLE_OUTPUT_PATH}/${TEST_NAME}")
-    add_executable(${TEST_NAME} "${REL_TEST_NAME}.cc")
-    target_link_libraries(${TEST_NAME} ${PYARROW_TEST_LINK_LIBS})
-  else()
-    # No executable, just invoke the test (probably a script) directly.
-    set(TEST_PATH ${CMAKE_CURRENT_SOURCE_DIR}/${REL_TEST_NAME})
-  endif()
-
-  add_test(${TEST_NAME}
-    ${BUILD_SUPPORT_DIR}/run-test.sh ${TEST_PATH})
-  if(ARGN)
-    set_tests_properties(${TEST_NAME} PROPERTIES ${ARGN})
-  endif()
-endfunction()
-
-# A wrapper for add_dependencies() that is compatible with NO_TESTS.
-function(ADD_PYARROW_TEST_DEPENDENCIES REL_TEST_NAME)
-  if(NO_TESTS)
-    return()
-  endif()
-  get_filename_component(TEST_NAME ${REL_TEST_NAME} NAME_WE)
-
-  add_dependencies(${TEST_NAME} ${ARGN})
-endfunction()
-
-enable_testing()
-
-############################################################
 # Dependencies
 ############################################################
-function(ADD_THIRDPARTY_LIB LIB_NAME)
-  set(options)
-  set(one_value_args SHARED_LIB STATIC_LIB)
-  set(multi_value_args DEPS)
-  cmake_parse_arguments(ARG "${options}" "${one_value_args}" "${multi_value_args}" ${ARGN})
-  if(ARG_UNPARSED_ARGUMENTS)
-    message(SEND_ERROR "Error: unrecognized arguments: ${ARG_UNPARSED_ARGUMENTS}")
-  endif()
-
-  if(("${PYARROW_LINK}" STREQUAL "s" AND ARG_STATIC_LIB) OR (NOT ARG_SHARED_LIB))
-    if(NOT ARG_STATIC_LIB)
-      message(FATAL_ERROR "No static or shared library provided for ${LIB_NAME}")
-    endif()
-    add_library(${LIB_NAME} STATIC IMPORTED)
-    set_target_properties(${LIB_NAME}
-      PROPERTIES IMPORTED_LOCATION "${ARG_STATIC_LIB}")
-    message(STATUS "Added static library dependency ${LIB_NAME}: ${ARG_STATIC_LIB}")
-  else()
-    add_library(${LIB_NAME} SHARED IMPORTED)
-    set_target_properties(${LIB_NAME}
-      PROPERTIES IMPORTED_LOCATION "${ARG_SHARED_LIB}")
-    message(STATUS "Added shared library dependency ${LIB_NAME}: ${ARG_SHARED_LIB}")
-  endif()
-
-  if(ARG_DEPS)
-    set_target_properties(${LIB_NAME}
-      PROPERTIES IMPORTED_LINK_INTERFACE_LIBRARIES "${ARG_DEPS}")
-  endif()
-
-  # Set up an "exported variant" for this thirdparty library (see "Visibility"
-  # above). It's the same as the real target, just with an "_exported" suffix.
-  # We prefer the static archive if it exists (as it's akin to an "internal"
-  # library), but we'll settle for the shared object if we must.
-  #
-  # A shared object exported variant will force any "leaf" library that
-  # transitively depends on it to also depend on it at runtime; this is
-  # desirable for some libraries (e.g. cyrus_sasl).
-  set(LIB_NAME_EXPORTED ${LIB_NAME}_exported)
-  if(ARG_STATIC_LIB)
-    add_library(${LIB_NAME_EXPORTED} STATIC IMPORTED)
-    set_target_properties(${LIB_NAME_EXPORTED}
-      PROPERTIES IMPORTED_LOCATION "${ARG_STATIC_LIB}")
-  else()
-    add_library(${LIB_NAME_EXPORTED} SHARED IMPORTED)
-    set_target_properties(${LIB_NAME_EXPORTED}
-      PROPERTIES IMPORTED_LOCATION "${ARG_SHARED_LIB}")
-  endif()
-  if(ARG_DEPS)
-    set_target_properties(${LIB_NAME_EXPORTED}
-      PROPERTIES IMPORTED_LINK_INTERFACE_LIBRARIES "${ARG_DEPS}")
-  endif()
-endfunction()
-
-## GMock
-if (PYARROW_BUILD_TESTS)
-  find_package(GTest REQUIRED)
-  include_directories(SYSTEM ${GTEST_INCLUDE_DIR})
-  ADD_THIRDPARTY_LIB(gtest
-    STATIC_LIB ${GTEST_STATIC_LIB})
-endif()
 
 ## Parquet
 find_package(Parquet)
@@ -352,6 +242,8 @@ if (PYARROW_BUNDLE_ARROW_CPP)
       COPYONLY)
   SET(ARROW_IPC_SHARED_LIB
       ${BUILD_OUTPUT_ROOT_DIRECTORY}/libarrow_ipc${CMAKE_SHARED_LIBRARY_SUFFIX})
+  SET(ARROW_PYTHON_SHARED_LIB
+      ${BUILD_OUTPUT_ROOT_DIRECTORY}/libarrow_python${CMAKE_SHARED_LIBRARY_SUFFIX})
 endif()
 
 ADD_THIRDPARTY_LIB(arrow
@@ -360,66 +252,8 @@ ADD_THIRDPARTY_LIB(arrow_io
   SHARED_LIB ${ARROW_IO_SHARED_LIB})
 ADD_THIRDPARTY_LIB(arrow_ipc
   SHARED_LIB ${ARROW_IPC_SHARED_LIB})
-
-############################################################
-# Linker setup
-############################################################
-
-set(PYARROW_MIN_TEST_LIBS
-  pyarrow_test_main
-  pyarrow)
-
-set(PYARROW_MIN_TEST_LIBS
-  pyarrow_test_main
-  pyarrow
-  ${PYARROW_BASE_LIBS})
-
-if(NOT APPLE AND PYARROW_BUILD_TESTS)
-  ADD_THIRDPARTY_LIB(python
-    SHARED_LIB "${PYTHON_LIBRARIES}")
-  list(APPEND PYARROW_MIN_TEST_LIBS python)
-endif()
-
-set(PYARROW_TEST_LINK_LIBS ${PYARROW_MIN_TEST_LIBS})
-
-############################################################
-# "make ctags" target
-############################################################
-if (UNIX)
-  add_custom_target(ctags ctags -R --languages=c++,c --exclude=thirdparty/installed)
-endif (UNIX)
-
-############################################################
-# "make etags" target
-############################################################
-if (UNIX)
-  add_custom_target(tags etags --members --declarations
-  `find ${CMAKE_CURRENT_SOURCE_DIR}/src
-   -name \\*.cc -or -name \\*.hh -or -name \\*.cpp -or -name \\*.h -or -name \\*.c -or
-   -name \\*.f`)
-  add_custom_target(etags DEPENDS tags)
-endif (UNIX)
-
-############################################################
-# "make cscope" target
-############################################################
-if (UNIX)
-  add_custom_target(cscope find ${CMAKE_CURRENT_SOURCE_DIR}
-  ( -name \\*.cc -or -name \\*.hh -or -name \\*.cpp -or
-    -name \\*.h -or -name \\*.c -or -name \\*.f )
-  -exec echo \"{}\" \; > cscope.files && cscope -q -b VERBATIM)
-endif (UNIX)
-
-############################################################
-# "make lint" target
-############################################################
-if (UNIX)
-  # Full lint
-  add_custom_target(lint ${BUILD_SUPPORT_DIR}/cpplint.py
-  --verbose=2
-  --filter=-whitespace/comments,-readability/todo,-build/header_guard
-    `find ${CMAKE_CURRENT_SOURCE_DIR}/src -name \\*.cc -or -name \\*.h`)
-endif (UNIX)
+ADD_THIRDPARTY_LIB(arrow_python
+  SHARED_LIB ${ARROW_PYTHON_SHARED_LIB})
 
 ############################################################
 # Subdirectories
@@ -429,9 +263,6 @@ if (UNIX)
   set(CMAKE_BUILD_WITH_INSTALL_RPATH TRUE)
 endif()
 
-add_subdirectory(src/pyarrow)
-add_subdirectory(src/pyarrow/util)
-
 set(CYTHON_EXTENSIONS
   array
   config
@@ -444,19 +275,11 @@ set(CYTHON_EXTENSIONS
   table
 )
 
-set(PYARROW_SRCS
-  src/pyarrow/common.cc
-  src/pyarrow/config.cc
-  src/pyarrow/helpers.cc
-  src/pyarrow/io.cc
-  src/pyarrow/adapters/builtin.cc
-  src/pyarrow/adapters/pandas.cc
-)
-
 set(LINK_LIBS
-  arrow
-  arrow_io
-  arrow_ipc
+  arrow_shared
+  arrow_io_shared
+  arrow_ipc_shared
+  arrow_python_shared
 )
 
 if (PYARROW_BUILD_PARQUET)
@@ -497,24 +320,12 @@ if (PYARROW_BUILD_JEMALLOC)
     SHARED_LIB ${ARROW_JEMALLOC_SHARED_LIB})
   set(LINK_LIBS
     ${LINK_LIBS}
-    arrow_jemalloc)
+    arrow_jemalloc_shared)
   set(CYTHON_EXTENSIONS
     ${CYTHON_EXTENSIONS}
     jemalloc)
 endif()
 
-add_library(pyarrow SHARED
-  ${PYARROW_SRCS})
-if (PYARROW_BUNDLE_ARROW_CPP)
-  set_target_properties(pyarrow PROPERTIES
-      INSTALL_RPATH "\$ORIGIN")
-endif()
-target_link_libraries(pyarrow ${LINK_LIBS})
-
-if(APPLE)
-  set_target_properties(pyarrow PROPERTIES LINK_FLAGS "-undefined dynamic_lookup")
-endif()
-
 ############################################################
 # Setup and build Cython modules
 ############################################################
@@ -555,5 +366,5 @@ foreach(module ${CYTHON_EXTENSIONS})
 
     set_target_properties(${module_name} PROPERTIES
       INSTALL_RPATH ${module_install_rpath})
-    target_link_libraries(${module_name} pyarrow)
+    target_link_libraries(${module_name} ${LINK_LIBS})
 endforeach(module)

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/cmake_modules/FindArrow.cmake
----------------------------------------------------------------------
diff --git a/python/cmake_modules/FindArrow.cmake b/python/cmake_modules/FindArrow.cmake
index 5d0207d..5030c9c 100644
--- a/python/cmake_modules/FindArrow.cmake
+++ b/python/cmake_modules/FindArrow.cmake
@@ -57,12 +57,18 @@ find_library(ARROW_JEMALLOC_LIB_PATH NAMES arrow_jemalloc
   ${ARROW_SEARCH_LIB_PATH}
   NO_DEFAULT_PATH)
 
+find_library(ARROW_PYTHON_LIB_PATH NAMES arrow_python
+  PATHS
+  ${ARROW_SEARCH_LIB_PATH}
+  NO_DEFAULT_PATH)
+
 if (ARROW_INCLUDE_DIR AND ARROW_LIB_PATH)
   set(ARROW_FOUND TRUE)
   set(ARROW_LIB_NAME libarrow)
   set(ARROW_IO_LIB_NAME libarrow_io)
   set(ARROW_IPC_LIB_NAME libarrow_ipc)
   set(ARROW_JEMALLOC_LIB_NAME libarrow_jemalloc)
+  set(ARROW_PYTHON_LIB_NAME libarrow_python)
 
   set(ARROW_LIBS ${ARROW_SEARCH_LIB_PATH})
   set(ARROW_STATIC_LIB ${ARROW_SEARCH_LIB_PATH}/${ARROW_LIB_NAME}.a)
@@ -77,6 +83,9 @@ if (ARROW_INCLUDE_DIR AND ARROW_LIB_PATH)
   set(ARROW_JEMALLOC_STATIC_LIB ${ARROW_SEARCH_LIB_PATH}/${ARROW_JEMALLOC_LIB_NAME}.a)
   set(ARROW_JEMALLOC_SHARED_LIB ${ARROW_LIBS}/${ARROW_JEMALLOC_LIB_NAME}${CMAKE_SHARED_LIBRARY_SUFFIX})
 
+  set(ARROW_PYTHON_STATIC_LIB ${ARROW_SEARCH_LIB_PATH}/${ARROW_PYTHON_LIB_NAME}.a)
+  set(ARROW_PYTHON_SHARED_LIB ${ARROW_LIBS}/${ARROW_PYTHON_LIB_NAME}${CMAKE_SHARED_LIBRARY_SUFFIX})
+
   if (NOT Arrow_FIND_QUIETLY)
     message(STATUS "Found the Arrow core library: ${ARROW_LIB_PATH}")
     message(STATUS "Found the Arrow IO library: ${ARROW_IO_LIB_PATH}")

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/cmake_modules/FindNumPy.cmake
----------------------------------------------------------------------
diff --git a/python/cmake_modules/FindNumPy.cmake b/python/cmake_modules/FindNumPy.cmake
deleted file mode 100644
index 58bb531..0000000
--- a/python/cmake_modules/FindNumPy.cmake
+++ /dev/null
@@ -1,100 +0,0 @@
-# - Find the NumPy libraries
-# This module finds if NumPy is installed, and sets the following variables
-# indicating where it is.
-#
-# TODO: Update to provide the libraries and paths for linking npymath lib.
-#
-#  NUMPY_FOUND               - was NumPy found
-#  NUMPY_VERSION             - the version of NumPy found as a string
-#  NUMPY_VERSION_MAJOR       - the major version number of NumPy
-#  NUMPY_VERSION_MINOR       - the minor version number of NumPy
-#  NUMPY_VERSION_PATCH       - the patch version number of NumPy
-#  NUMPY_VERSION_DECIMAL     - e.g. version 1.6.1 is 10601
-#  NUMPY_INCLUDE_DIRS        - path to the NumPy include files
-
-#============================================================================
-# Copyright 2012 Continuum Analytics, Inc.
-#
-# MIT License
-#
-# Permission is hereby granted, free of charge, to any person obtaining
-# a copy of this software and associated documentation files
-# (the "Software"), to deal in the Software without restriction, including
-# without limitation the rights to use, copy, modify, merge, publish,
-# distribute, sublicense, and/or sell copies of the Software, and to permit
-# persons to whom the Software is furnished to do so, subject to
-# the following conditions:
-#
-# The above copyright notice and this permission notice shall be included
-# in all copies or substantial portions of the Software.
-#
-# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
-# OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
-# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL
-# THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR
-# OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE,
-# ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
-# OTHER DEALINGS IN THE SOFTWARE.
-#
-#============================================================================
-
-# Finding NumPy involves calling the Python interpreter
-if(NumPy_FIND_REQUIRED)
-    find_package(PythonInterp REQUIRED)
-else()
-    find_package(PythonInterp)
-endif()
-
-if(NOT PYTHONINTERP_FOUND)
-    set(NUMPY_FOUND FALSE)
-    return()
-endif()
-
-execute_process(COMMAND "${PYTHON_EXECUTABLE}" "-c"
-    "import numpy as n; print(n.__version__); print(n.get_include());"
-    RESULT_VARIABLE _NUMPY_SEARCH_SUCCESS
-    OUTPUT_VARIABLE _NUMPY_VALUES_OUTPUT
-    ERROR_VARIABLE _NUMPY_ERROR_VALUE
-    OUTPUT_STRIP_TRAILING_WHITESPACE)
-
-if(NOT _NUMPY_SEARCH_SUCCESS MATCHES 0)
-    if(NumPy_FIND_REQUIRED)
-        message(FATAL_ERROR
-            "NumPy import failure:\n${_NUMPY_ERROR_VALUE}")
-    endif()
-    set(NUMPY_FOUND FALSE)
-    return()
-endif()
-
-# Convert the process output into a list
-string(REGEX REPLACE ";" "\\\\;" _NUMPY_VALUES ${_NUMPY_VALUES_OUTPUT})
-string(REGEX REPLACE "\n" ";" _NUMPY_VALUES ${_NUMPY_VALUES})
-list(GET _NUMPY_VALUES 0 NUMPY_VERSION)
-list(GET _NUMPY_VALUES 1 NUMPY_INCLUDE_DIRS)
-
-string(REGEX MATCH "^[0-9]+\\.[0-9]+\\.[0-9]+" _VER_CHECK "${NUMPY_VERSION}")
-if("${_VER_CHECK}" STREQUAL "")
-    # The output from Python was unexpected. Raise an error always
-    # here, because we found NumPy, but it appears to be corrupted somehow.
-    message(FATAL_ERROR
-        "Requested version and include path from NumPy, got instead:\n${_NUMPY_VALUES_OUTPUT}\n")
-    return()
-endif()
-
-# Make sure all directory separators are '/'
-string(REGEX REPLACE "\\\\" "/" NUMPY_INCLUDE_DIRS ${NUMPY_INCLUDE_DIRS})
-
-# Get the major and minor version numbers
-string(REGEX REPLACE "\\." ";" _NUMPY_VERSION_LIST ${NUMPY_VERSION})
-list(GET _NUMPY_VERSION_LIST 0 NUMPY_VERSION_MAJOR)
-list(GET _NUMPY_VERSION_LIST 1 NUMPY_VERSION_MINOR)
-list(GET _NUMPY_VERSION_LIST 2 NUMPY_VERSION_PATCH)
-string(REGEX MATCH "[0-9]*" NUMPY_VERSION_PATCH ${NUMPY_VERSION_PATCH})
-math(EXPR NUMPY_VERSION_DECIMAL
-    "(${NUMPY_VERSION_MAJOR} * 10000) + (${NUMPY_VERSION_MINOR} * 100) + ${NUMPY_VERSION_PATCH}")
-
-find_package_message(NUMPY
-    "Found NumPy: version \"${NUMPY_VERSION}\" ${NUMPY_INCLUDE_DIRS}"
-    "${NUMPY_INCLUDE_DIRS}${NUMPY_VERSION}")
-
-set(NUMPY_FOUND TRUE)

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/cmake_modules/FindPythonLibsNew.cmake
----------------------------------------------------------------------
diff --git a/python/cmake_modules/FindPythonLibsNew.cmake b/python/cmake_modules/FindPythonLibsNew.cmake
deleted file mode 100644
index 1000a95..0000000
--- a/python/cmake_modules/FindPythonLibsNew.cmake
+++ /dev/null
@@ -1,241 +0,0 @@
-# - Find python libraries
-# This module finds the libraries corresponding to the Python interpeter
-# FindPythonInterp provides.
-# This code sets the following variables:
-#
-#  PYTHONLIBS_FOUND           - have the Python libs been found
-#  PYTHON_PREFIX              - path to the Python installation
-#  PYTHON_LIBRARIES           - path to the python library
-#  PYTHON_INCLUDE_DIRS        - path to where Python.h is found
-#  PYTHON_SITE_PACKAGES       - path to installation site-packages
-#  PYTHON_IS_DEBUG            - whether the Python interpreter is a debug build
-#
-#  PYTHON_INCLUDE_PATH        - path to where Python.h is found (deprecated)
-#
-# A function PYTHON_ADD_MODULE(<name> src1 src2 ... srcN) is defined
-# to build modules for python.
-#
-# Thanks to talljimbo for the patch adding the 'LDVERSION' config
-# variable usage.
-
-#=============================================================================
-# Copyright 2001-2009 Kitware, Inc.
-# Copyright 2012-2014 Continuum Analytics, Inc.
-#
-# All rights reserved.
-#
-# Redistribution and use in source and binary forms, with or without
-# modification, are permitted provided that the following conditions
-# are met:
-#
-# * Redistributions of source code must retain the above copyright
-# notice, this list of conditions and the following disclaimer.
-#
-# * Redistributions in binary form must reproduce the above copyright
-# notice, this list of conditions and the following disclaimer in the
-# documentation and/or other materials provided with the distribution.
-#
-# * Neither the names of Kitware, Inc., the Insight Software Consortium,
-# nor the names of their contributors may be used to endorse or promote
-# products derived from this software without specific prior written
-# permission.
-#
-# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
-# "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
-# LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
-# # A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
-# HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
-# SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
-# LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
-# DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
-# THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
-# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
-# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
-#=============================================================================
-# (To distribute this file outside of CMake, substitute the full
-#  License text for the above reference.)
-
-# Use the Python interpreter to find the libs.
-if(PythonLibsNew_FIND_REQUIRED)
-    find_package(PythonInterp REQUIRED)
-else()
-    find_package(PythonInterp)
-endif()
-
-if(NOT PYTHONINTERP_FOUND)
-    set(PYTHONLIBS_FOUND FALSE)
-    return()
-endif()
-
-# According to http://stackoverflow.com/questions/646518/python-how-to-detect-debug-interpreter
-# testing whether sys has the gettotalrefcount function is a reliable,
-# cross-platform way to detect a CPython debug interpreter.
-#
-# The library suffix is from the config var LDVERSION sometimes, otherwise
-# VERSION. VERSION will typically be like "2.7" on unix, and "27" on windows.
-#
-# The config var LIBPL is for Linux, and helps on Debian Jessie where the
-# addition of multi-arch support shuffled things around.
-execute_process(COMMAND "${PYTHON_EXECUTABLE}" "-c"
-    "from distutils import sysconfig as s;import sys;import struct;
-print('.'.join(str(v) for v in sys.version_info));
-print(sys.prefix);
-print(s.get_python_inc(plat_specific=True));
-print(s.get_python_lib(plat_specific=True));
-print(s.get_config_var('SO'));
-print(hasattr(sys, 'gettotalrefcount')+0);
-print(struct.calcsize('@P'));
-print(s.get_config_var('LDVERSION') or s.get_config_var('VERSION'));
-print(s.get_config_var('LIBPL'));
-"
-    RESULT_VARIABLE _PYTHON_SUCCESS
-    OUTPUT_VARIABLE _PYTHON_VALUES
-    ERROR_VARIABLE _PYTHON_ERROR_VALUE
-    OUTPUT_STRIP_TRAILING_WHITESPACE)
-
-if(NOT _PYTHON_SUCCESS MATCHES 0)
-    if(PythonLibsNew_FIND_REQUIRED)
-        message(FATAL_ERROR
-            "Python config failure:\n${_PYTHON_ERROR_VALUE}")
-    endif()
-    set(PYTHONLIBS_FOUND FALSE)
-    return()
-endif()
-
-# Convert the process output into a list
-string(REGEX REPLACE ";" "\\\\;" _PYTHON_VALUES ${_PYTHON_VALUES})
-string(REGEX REPLACE "\n" ";" _PYTHON_VALUES ${_PYTHON_VALUES})
-list(GET _PYTHON_VALUES 0 _PYTHON_VERSION_LIST)
-list(GET _PYTHON_VALUES 1 PYTHON_PREFIX)
-list(GET _PYTHON_VALUES 2 PYTHON_INCLUDE_DIR)
-list(GET _PYTHON_VALUES 3 PYTHON_SITE_PACKAGES)
-list(GET _PYTHON_VALUES 4 PYTHON_MODULE_EXTENSION)
-list(GET _PYTHON_VALUES 5 PYTHON_IS_DEBUG)
-list(GET _PYTHON_VALUES 6 PYTHON_SIZEOF_VOID_P)
-list(GET _PYTHON_VALUES 7 PYTHON_LIBRARY_SUFFIX)
-list(GET _PYTHON_VALUES 8 PYTHON_LIBRARY_PATH)
-
-# Make sure the Python has the same pointer-size as the chosen compiler
-# Skip the check on OS X, it doesn't consistently have CMAKE_SIZEOF_VOID_P defined
-if((NOT APPLE) AND (NOT "${PYTHON_SIZEOF_VOID_P}" STREQUAL "${CMAKE_SIZEOF_VOID_P}"))
-    if(PythonLibsNew_FIND_REQUIRED)
-        math(EXPR _PYTHON_BITS "${PYTHON_SIZEOF_VOID_P} * 8")
-        math(EXPR _CMAKE_BITS "${CMAKE_SIZEOF_VOID_P} * 8")
-        message(FATAL_ERROR
-            "Python config failure: Python is ${_PYTHON_BITS}-bit, "
-            "chosen compiler is  ${_CMAKE_BITS}-bit")
-    endif()
-    set(PYTHONLIBS_FOUND FALSE)
-    return()
-endif()
-
-# The built-in FindPython didn't always give the version numbers
-string(REGEX REPLACE "\\." ";" _PYTHON_VERSION_LIST ${_PYTHON_VERSION_LIST})
-list(GET _PYTHON_VERSION_LIST 0 PYTHON_VERSION_MAJOR)
-list(GET _PYTHON_VERSION_LIST 1 PYTHON_VERSION_MINOR)
-list(GET _PYTHON_VERSION_LIST 2 PYTHON_VERSION_PATCH)
-
-# Make sure all directory separators are '/'
-string(REGEX REPLACE "\\\\" "/" PYTHON_PREFIX ${PYTHON_PREFIX})
-string(REGEX REPLACE "\\\\" "/" PYTHON_INCLUDE_DIR ${PYTHON_INCLUDE_DIR})
-string(REGEX REPLACE "\\\\" "/" PYTHON_SITE_PACKAGES ${PYTHON_SITE_PACKAGES})
-
-if(CMAKE_HOST_WIN32)
-    if("${CMAKE_CXX_COMPILER_ID}" STREQUAL "MSVC")
-        set(PYTHON_LIBRARY
-            "${PYTHON_PREFIX}/libs/Python${PYTHON_LIBRARY_SUFFIX}.lib")
-    else()
-        set(PYTHON_LIBRARY "${PYTHON_PREFIX}/libs/libpython${PYTHON_LIBRARY_SUFFIX}.a")
-    endif()
-elseif(APPLE)
-     # Seems to require "-undefined dynamic_lookup" instead of linking
-     # against the .dylib, otherwise it crashes. This flag is added
-     # below
-    set(PYTHON_LIBRARY "")
-    #set(PYTHON_LIBRARY
-    #    "${PYTHON_PREFIX}/lib/libpython${PYTHON_LIBRARY_SUFFIX}.dylib")
-else()
-    if(${PYTHON_SIZEOF_VOID_P} MATCHES 8)
-        set(_PYTHON_LIBS_SEARCH "${PYTHON_PREFIX}/lib64" "${PYTHON_PREFIX}/lib" "${PYTHON_LIBRARY_PATH}")
-    else()
-        set(_PYTHON_LIBS_SEARCH "${PYTHON_PREFIX}/lib" "${PYTHON_LIBRARY_PATH}")
-    endif()
-    message(STATUS "Searching for Python libs in ${_PYTHON_LIBS_SEARCH}")
-    message(STATUS "Looking for python${PYTHON_LIBRARY_SUFFIX}")
-    # Probably this needs to be more involved. It would be nice if the config
-    # information the python interpreter itself gave us were more complete.
-    find_library(PYTHON_LIBRARY
-        NAMES "python${PYTHON_LIBRARY_SUFFIX}"
-        PATHS ${_PYTHON_LIBS_SEARCH}
-        NO_SYSTEM_ENVIRONMENT_PATH)
-    message(STATUS "Found Python lib ${PYTHON_LIBRARY}")
-endif()
-
-# For backward compatibility, set PYTHON_INCLUDE_PATH, but make it internal.
-SET(PYTHON_INCLUDE_PATH "${PYTHON_INCLUDE_DIR}" CACHE INTERNAL
-          "Path to where Python.h is found (deprecated)")
-
-MARK_AS_ADVANCED(
-  PYTHON_LIBRARY
-  PYTHON_INCLUDE_DIR
-)
-
-# We use PYTHON_INCLUDE_DIR, PYTHON_LIBRARY and PYTHON_DEBUG_LIBRARY for the
-# cache entries because they are meant to specify the location of a single
-# library. We now set the variables listed by the documentation for this
-# module.
-SET(PYTHON_INCLUDE_DIRS "${PYTHON_INCLUDE_DIR}")
-SET(PYTHON_LIBRARIES "${PYTHON_LIBRARY}")
-SET(PYTHON_DEBUG_LIBRARIES "${PYTHON_DEBUG_LIBRARY}")
-
-
-# Don't know how to get to this directory, just doing something simple :P
-#INCLUDE(${CMAKE_CURRENT_LIST_DIR}/FindPackageHandleStandardArgs.cmake)
-#FIND_PACKAGE_HANDLE_STANDARD_ARGS(PythonLibs DEFAULT_MSG PYTHON_LIBRARIES PYTHON_INCLUDE_DIRS)
-find_package_message(PYTHON
-    "Found PythonLibs: ${PYTHON_LIBRARY}"
-    "${PYTHON_EXECUTABLE}${PYTHON_VERSION}")
-
-
-# PYTHON_ADD_MODULE(<name> src1 src2 ... srcN) is used to build modules for python.
-FUNCTION(PYTHON_ADD_MODULE _NAME )
-  GET_PROPERTY(_TARGET_SUPPORTS_SHARED_LIBS
-    GLOBAL PROPERTY TARGET_SUPPORTS_SHARED_LIBS)
-  OPTION(PYTHON_ENABLE_MODULE_${_NAME} "Add module ${_NAME}" TRUE)
-  OPTION(PYTHON_MODULE_${_NAME}_BUILD_SHARED
-    "Add module ${_NAME} shared" ${_TARGET_SUPPORTS_SHARED_LIBS})
-
-  # Mark these options as advanced
-  MARK_AS_ADVANCED(PYTHON_ENABLE_MODULE_${_NAME}
-    PYTHON_MODULE_${_NAME}_BUILD_SHARED)
-
-  IF(PYTHON_ENABLE_MODULE_${_NAME})
-    IF(PYTHON_MODULE_${_NAME}_BUILD_SHARED)
-      SET(PY_MODULE_TYPE MODULE)
-    ELSE(PYTHON_MODULE_${_NAME}_BUILD_SHARED)
-      SET(PY_MODULE_TYPE STATIC)
-      SET_PROPERTY(GLOBAL  APPEND  PROPERTY  PY_STATIC_MODULES_LIST ${_NAME})
-    ENDIF(PYTHON_MODULE_${_NAME}_BUILD_SHARED)
-
-    SET_PROPERTY(GLOBAL  APPEND  PROPERTY  PY_MODULES_LIST ${_NAME})
-    ADD_LIBRARY(${_NAME} ${PY_MODULE_TYPE} ${ARGN})
-    IF(APPLE)
-      # On OS X, linking against the Python libraries causes
-      # segfaults, so do this dynamic lookup instead.
-      SET_TARGET_PROPERTIES(${_NAME} PROPERTIES LINK_FLAGS
-                          "-undefined dynamic_lookup")
-    ELSE()
-     # In general, we should not link against libpython as we do not embed
-     # the Python interpreter. The python binary itself can then define where
-     # the symbols should loaded from.
-     SET_TARGET_PROPERTIES(${_NAME} PROPERTIES LINK_FLAGS
-         "-Wl,-undefined,dynamic_lookup")
-    ENDIF()
-    IF(PYTHON_MODULE_${_NAME}_BUILD_SHARED)
-      SET_TARGET_PROPERTIES(${_NAME} PROPERTIES PREFIX "${PYTHON_MODULE_PREFIX}")
-      SET_TARGET_PROPERTIES(${_NAME} PROPERTIES SUFFIX "${PYTHON_MODULE_EXTENSION}")
-    ELSE()
-    ENDIF()
-
-  ENDIF(PYTHON_ENABLE_MODULE_${_NAME})
-ENDFUNCTION(PYTHON_ADD_MODULE)

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/pyarrow/config.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/config.pyx b/python/pyarrow/config.pyx
index 5ad7cf5..536f278 100644
--- a/python/pyarrow/config.pyx
+++ b/python/pyarrow/config.pyx
@@ -14,21 +14,21 @@
 # distutils: language = c++
 # cython: embedsignature = True
 
-cdef extern from 'pyarrow/do_import_numpy.h':
+cdef extern from 'arrow/python/do_import_numpy.h':
     pass
 
-cdef extern from 'pyarrow/numpy_interop.h' namespace 'arrow::py':
+cdef extern from 'arrow/python/numpy_interop.h' namespace 'arrow::py':
     int import_numpy()
 
-cdef extern from 'pyarrow/config.h' namespace 'arrow::py':
-    void pyarrow_init()
-    void pyarrow_set_numpy_nan(object o)
+cdef extern from 'arrow/python/config.h' namespace 'arrow::py':
+    void Init()
+    void set_numpy_nan(object o)
 
 import_numpy()
-pyarrow_init()
+Init()
 
 import numpy as np
-pyarrow_set_numpy_nan(np.nan)
+set_numpy_nan(np.nan)
 
 import multiprocessing
 import os

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/pyarrow/includes/pyarrow.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/pyarrow.pxd b/python/pyarrow/includes/pyarrow.pxd
index 3fdbebc..c3fdf4b 100644
--- a/python/pyarrow/includes/pyarrow.pxd
+++ b/python/pyarrow/includes/pyarrow.pxd
@@ -25,7 +25,7 @@ from pyarrow.includes.libarrow cimport (CArray, CBuffer, CColumn,
 cimport pyarrow.includes.libarrow_io as arrow_io
 
 
-cdef extern from "pyarrow/api.h" namespace "arrow::py" nogil:
+cdef extern from "arrow/python/api.h" namespace "arrow::py" nogil:
     shared_ptr[CDataType] GetPrimitiveType(Type type)
     shared_ptr[CDataType] GetTimestampType(TimeUnit unit)
     CStatus ConvertPySequence(object obj, CMemoryPool* pool,
@@ -53,13 +53,9 @@ cdef extern from "pyarrow/api.h" namespace "arrow::py" nogil:
     void set_default_memory_pool(CMemoryPool* pool)
     CMemoryPool* get_memory_pool()
 
-
-cdef extern from "pyarrow/common.h" namespace "arrow::py" nogil:
     cdef cppclass PyBuffer(CBuffer):
         PyBuffer(object o)
 
-
-cdef extern from "pyarrow/io.h" namespace "arrow::py" nogil:
     cdef cppclass PyReadableFile(arrow_io.RandomAccessFile):
         PyReadableFile(object fo)
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/setup.py
----------------------------------------------------------------------
diff --git a/python/setup.py b/python/setup.py
index 9abf985..dae6cb2 100644
--- a/python/setup.py
+++ b/python/setup.py
@@ -186,7 +186,7 @@ class build_ext(_build_ext):
             # a bit hacky
             build_lib = saved_cwd
 
-        # Move the built libpyarrow library to the place expected by the Python
+        # Move the libraries to the place expected by the Python
         # build
         shared_library_prefix = 'lib'
         if sys.platform == 'darwin':
@@ -203,15 +203,16 @@ class build_ext(_build_ext):
             pass
 
         def move_lib(lib_name):
-            lib_filename = shared_library_prefix + lib_name + shared_library_suffix
+            lib_filename = (shared_library_prefix + lib_name +
+                            shared_library_suffix)
             shutil.move(pjoin(self.build_type, lib_filename),
                         pjoin(build_lib, 'pyarrow', lib_filename))
 
-        move_lib("pyarrow")
         if self.bundle_arrow_cpp:
             move_lib("arrow")
             move_lib("arrow_io")
             move_lib("arrow_ipc")
+            move_lib("arrow_python")
             if self.with_jemalloc:
                 move_lib("arrow_jemalloc")
             if self.with_parquet:
@@ -227,14 +228,14 @@ class build_ext(_build_ext):
                 if self._failure_permitted(name):
                     print('Cython module {0} failure permitted'.format(name))
                     continue
-                raise RuntimeError('libpyarrow C-extension failed to build:',
+                raise RuntimeError('pyarrow C-extension failed to build:',
                                    os.path.abspath(built_path))
 
             ext_path = pjoin(build_lib, self._get_cmake_ext_path(name))
             if os.path.exists(ext_path):
                 os.remove(ext_path)
             self.mkpath(os.path.dirname(ext_path))
-            print('Moving built libpyarrow C-extension', built_path,
+            print('Moving built C-extension', built_path,
                   'to build path', ext_path)
             shutil.move(self.get_ext_built(name), ext_path)
             self._found_names.append(name)

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/CMakeLists.txt b/python/src/pyarrow/CMakeLists.txt
deleted file mode 100644
index 9e69718..0000000
--- a/python/src/pyarrow/CMakeLists.txt
+++ /dev/null
@@ -1,22 +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.
-
-#######################################
-# Unit tests
-#######################################
-
-ADD_PYARROW_TEST(adapters/pandas-test)

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/adapters/builtin.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/builtin.cc b/python/src/pyarrow/adapters/builtin.cc
deleted file mode 100644
index 06e098a..0000000
--- a/python/src/pyarrow/adapters/builtin.cc
+++ /dev/null
@@ -1,527 +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 <Python.h>
-#include <datetime.h>
-#include <sstream>
-
-#include "pyarrow/adapters/builtin.h"
-
-#include "arrow/api.h"
-#include "arrow/status.h"
-
-#include "pyarrow/helpers.h"
-#include "pyarrow/util/datetime.h"
-
-namespace arrow {
-namespace py {
-
-static inline bool IsPyInteger(PyObject* obj) {
-#if PYARROW_IS_PY2
-  return PyLong_Check(obj) || PyInt_Check(obj);
-#else
-  return PyLong_Check(obj);
-#endif
-}
-
-class ScalarVisitor {
- public:
-  ScalarVisitor()
-      : total_count_(0),
-        none_count_(0),
-        bool_count_(0),
-        int_count_(0),
-        date_count_(0),
-        timestamp_count_(0),
-        float_count_(0),
-        binary_count_(0),
-        unicode_count_(0) {}
-
-  void Visit(PyObject* obj) {
-    ++total_count_;
-    if (obj == Py_None) {
-      ++none_count_;
-    } else if (PyBool_Check(obj)) {
-      ++bool_count_;
-    } else if (PyFloat_Check(obj)) {
-      ++float_count_;
-    } else if (IsPyInteger(obj)) {
-      ++int_count_;
-    } else if (PyDate_CheckExact(obj)) {
-      ++date_count_;
-    } else if (PyDateTime_CheckExact(obj)) {
-      ++timestamp_count_;
-    } else if (PyBytes_Check(obj)) {
-      ++binary_count_;
-    } else if (PyUnicode_Check(obj)) {
-      ++unicode_count_;
-    } else {
-      // TODO(wesm): accumulate error information somewhere
-    }
-  }
-
-  std::shared_ptr<DataType> GetType() {
-    // TODO(wesm): handling mixed-type cases
-    if (float_count_) {
-      return float64();
-    } else if (int_count_) {
-      // TODO(wesm): tighter type later
-      return int64();
-    } else if (date_count_) {
-      return date64();
-    } else if (timestamp_count_) {
-      return timestamp(TimeUnit::MICRO);
-    } else if (bool_count_) {
-      return boolean();
-    } else if (binary_count_) {
-      return binary();
-    } else if (unicode_count_) {
-      return utf8();
-    } else {
-      return null();
-    }
-  }
-
-  int64_t total_count() const { return total_count_; }
-
- private:
-  int64_t total_count_;
-  int64_t none_count_;
-  int64_t bool_count_;
-  int64_t int_count_;
-  int64_t date_count_;
-  int64_t timestamp_count_;
-  int64_t float_count_;
-  int64_t binary_count_;
-  int64_t unicode_count_;
-
-  // Place to accumulate errors
-  // std::vector<Status> errors_;
-};
-
-static constexpr int MAX_NESTING_LEVELS = 32;
-
-class SeqVisitor {
- public:
-  SeqVisitor() : max_nesting_level_(0) {
-    memset(nesting_histogram_, 0, MAX_NESTING_LEVELS * sizeof(int));
-  }
-
-  Status Visit(PyObject* obj, int level = 0) {
-    Py_ssize_t size = PySequence_Size(obj);
-
-    if (level > max_nesting_level_) { max_nesting_level_ = level; }
-
-    for (int64_t i = 0; i < size; ++i) {
-      // TODO(wesm): Error checking?
-      // TODO(wesm): Specialize for PyList_GET_ITEM?
-      OwnedRef item_ref(PySequence_GetItem(obj, i));
-      PyObject* item = item_ref.obj();
-
-      if (PyList_Check(item)) {
-        RETURN_NOT_OK(Visit(item, level + 1));
-      } else if (PyDict_Check(item)) {
-        return Status::NotImplemented("No type inference for dicts");
-      } else {
-        // We permit nulls at any level of nesting
-        if (item == Py_None) {
-          // TODO
-        } else {
-          ++nesting_histogram_[level];
-          scalars_.Visit(item);
-        }
-      }
-    }
-    return Status::OK();
-  }
-
-  std::shared_ptr<DataType> GetType() {
-    if (scalars_.total_count() == 0) {
-      if (max_nesting_level_ == 0) {
-        return null();
-      } else {
-        return nullptr;
-      }
-    } else {
-      std::shared_ptr<DataType> result = scalars_.GetType();
-      for (int i = 0; i < max_nesting_level_; ++i) {
-        result = std::make_shared<ListType>(result);
-      }
-      return result;
-    }
-  }
-
-  Status Validate() const {
-    if (scalars_.total_count() > 0) {
-      if (num_nesting_levels() > 1) {
-        return Status::Invalid("Mixed nesting levels not supported");
-      } else if (max_observed_level() < max_nesting_level_) {
-        return Status::Invalid("Mixed nesting levels not supported");
-      }
-    }
-    return Status::OK();
-  }
-
-  int max_observed_level() const {
-    int result = 0;
-    for (int i = 0; i < MAX_NESTING_LEVELS; ++i) {
-      if (nesting_histogram_[i] > 0) { result = i; }
-    }
-    return result;
-  }
-
-  int num_nesting_levels() const {
-    int result = 0;
-    for (int i = 0; i < MAX_NESTING_LEVELS; ++i) {
-      if (nesting_histogram_[i] > 0) { ++result; }
-    }
-    return result;
-  }
-
- private:
-  ScalarVisitor scalars_;
-
-  // Track observed
-  int max_nesting_level_;
-  int nesting_histogram_[MAX_NESTING_LEVELS];
-};
-
-// Non-exhaustive type inference
-Status InferArrowType(PyObject* obj, int64_t* size, std::shared_ptr<DataType>* out_type) {
-  *size = PySequence_Size(obj);
-  if (PyErr_Occurred()) {
-    // Not a sequence
-    PyErr_Clear();
-    return Status::TypeError("Object is not a sequence");
-  }
-
-  // For 0-length sequences, refuse to guess
-  if (*size == 0) { *out_type = null(); }
-
-  SeqVisitor seq_visitor;
-  RETURN_NOT_OK(seq_visitor.Visit(obj));
-  RETURN_NOT_OK(seq_visitor.Validate());
-
-  *out_type = seq_visitor.GetType();
-
-  if (*out_type == nullptr) { return Status::TypeError("Unable to determine data type"); }
-
-  return Status::OK();
-}
-
-// Marshal Python sequence (list, tuple, etc.) to Arrow array
-class SeqConverter {
- public:
-  virtual Status Init(const std::shared_ptr<ArrayBuilder>& builder) {
-    builder_ = builder;
-    return Status::OK();
-  }
-
-  virtual Status AppendData(PyObject* seq) = 0;
-
- protected:
-  std::shared_ptr<ArrayBuilder> builder_;
-};
-
-template <typename BuilderType>
-class TypedConverter : public SeqConverter {
- public:
-  Status Init(const std::shared_ptr<ArrayBuilder>& builder) override {
-    builder_ = builder;
-    typed_builder_ = static_cast<BuilderType*>(builder.get());
-    return Status::OK();
-  }
-
- protected:
-  BuilderType* typed_builder_;
-};
-
-class BoolConverter : public TypedConverter<BooleanBuilder> {
- public:
-  Status AppendData(PyObject* seq) override {
-    Py_ssize_t size = PySequence_Size(seq);
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    for (int64_t i = 0; i < size; ++i) {
-      OwnedRef item(PySequence_GetItem(seq, i));
-      if (item.obj() == Py_None) {
-        typed_builder_->AppendNull();
-      } else {
-        if (item.obj() == Py_True) {
-          typed_builder_->Append(true);
-        } else {
-          typed_builder_->Append(false);
-        }
-      }
-    }
-    return Status::OK();
-  }
-};
-
-class Int64Converter : public TypedConverter<Int64Builder> {
- public:
-  Status AppendData(PyObject* seq) override {
-    int64_t val;
-    Py_ssize_t size = PySequence_Size(seq);
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    for (int64_t i = 0; i < size; ++i) {
-      OwnedRef item(PySequence_GetItem(seq, i));
-      if (item.obj() == Py_None) {
-        typed_builder_->AppendNull();
-      } else {
-        val = PyLong_AsLongLong(item.obj());
-        RETURN_IF_PYERROR();
-        typed_builder_->Append(val);
-      }
-    }
-    return Status::OK();
-  }
-};
-
-class DateConverter : public TypedConverter<Date64Builder> {
- public:
-  Status AppendData(PyObject* seq) override {
-    Py_ssize_t size = PySequence_Size(seq);
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    for (int64_t i = 0; i < size; ++i) {
-      OwnedRef item(PySequence_GetItem(seq, i));
-      if (item.obj() == Py_None) {
-        typed_builder_->AppendNull();
-      } else {
-        PyDateTime_Date* pydate = reinterpret_cast<PyDateTime_Date*>(item.obj());
-        typed_builder_->Append(PyDate_to_ms(pydate));
-      }
-    }
-    return Status::OK();
-  }
-};
-
-class TimestampConverter : public TypedConverter<TimestampBuilder> {
- public:
-  Status AppendData(PyObject* seq) override {
-    Py_ssize_t size = PySequence_Size(seq);
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    for (int64_t i = 0; i < size; ++i) {
-      OwnedRef item(PySequence_GetItem(seq, i));
-      if (item.obj() == Py_None) {
-        typed_builder_->AppendNull();
-      } else {
-        PyDateTime_DateTime* pydatetime =
-            reinterpret_cast<PyDateTime_DateTime*>(item.obj());
-        struct tm datetime = {0};
-        datetime.tm_year = PyDateTime_GET_YEAR(pydatetime) - 1900;
-        datetime.tm_mon = PyDateTime_GET_MONTH(pydatetime) - 1;
-        datetime.tm_mday = PyDateTime_GET_DAY(pydatetime);
-        datetime.tm_hour = PyDateTime_DATE_GET_HOUR(pydatetime);
-        datetime.tm_min = PyDateTime_DATE_GET_MINUTE(pydatetime);
-        datetime.tm_sec = PyDateTime_DATE_GET_SECOND(pydatetime);
-        int us = PyDateTime_DATE_GET_MICROSECOND(pydatetime);
-        RETURN_IF_PYERROR();
-        struct tm epoch = {0};
-        epoch.tm_year = 70;
-        epoch.tm_mday = 1;
-        // Microseconds since the epoch
-        int64_t val = lrint(difftime(mktime(&datetime), mktime(&epoch))) * 1000000 + us;
-        typed_builder_->Append(val);
-      }
-    }
-    return Status::OK();
-  }
-};
-
-class DoubleConverter : public TypedConverter<DoubleBuilder> {
- public:
-  Status AppendData(PyObject* seq) override {
-    double val;
-    Py_ssize_t size = PySequence_Size(seq);
-    RETURN_NOT_OK(typed_builder_->Reserve(size));
-    for (int64_t i = 0; i < size; ++i) {
-      OwnedRef item(PySequence_GetItem(seq, i));
-      if (item.obj() == Py_None) {
-        typed_builder_->AppendNull();
-      } else {
-        val = PyFloat_AsDouble(item.obj());
-        RETURN_IF_PYERROR();
-        typed_builder_->Append(val);
-      }
-    }
-    return Status::OK();
-  }
-};
-
-class BytesConverter : public TypedConverter<BinaryBuilder> {
- public:
-  Status AppendData(PyObject* seq) override {
-    PyObject* item;
-    PyObject* bytes_obj;
-    OwnedRef tmp;
-    const char* bytes;
-    int64_t length;
-    Py_ssize_t size = PySequence_Size(seq);
-    for (int64_t i = 0; i < size; ++i) {
-      item = PySequence_GetItem(seq, i);
-      OwnedRef holder(item);
-
-      if (item == Py_None) {
-        RETURN_NOT_OK(typed_builder_->AppendNull());
-        continue;
-      } else if (PyUnicode_Check(item)) {
-        tmp.reset(PyUnicode_AsUTF8String(item));
-        RETURN_IF_PYERROR();
-        bytes_obj = tmp.obj();
-      } else if (PyBytes_Check(item)) {
-        bytes_obj = item;
-      } else {
-        return Status::TypeError("Non-string value encountered");
-      }
-      // No error checking
-      length = PyBytes_GET_SIZE(bytes_obj);
-      bytes = PyBytes_AS_STRING(bytes_obj);
-      RETURN_NOT_OK(typed_builder_->Append(bytes, length));
-    }
-    return Status::OK();
-  }
-};
-
-class UTF8Converter : public TypedConverter<StringBuilder> {
- public:
-  Status AppendData(PyObject* seq) override {
-    PyObject* item;
-    PyObject* bytes_obj;
-    OwnedRef tmp;
-    const char* bytes;
-    int64_t length;
-    Py_ssize_t size = PySequence_Size(seq);
-    for (int64_t i = 0; i < size; ++i) {
-      item = PySequence_GetItem(seq, i);
-      OwnedRef holder(item);
-
-      if (item == Py_None) {
-        RETURN_NOT_OK(typed_builder_->AppendNull());
-        continue;
-      } else if (!PyUnicode_Check(item)) {
-        return Status::TypeError("Non-unicode value encountered");
-      }
-      tmp.reset(PyUnicode_AsUTF8String(item));
-      RETURN_IF_PYERROR();
-      bytes_obj = tmp.obj();
-
-      // No error checking
-      length = PyBytes_GET_SIZE(bytes_obj);
-      bytes = PyBytes_AS_STRING(bytes_obj);
-      RETURN_NOT_OK(typed_builder_->Append(bytes, length));
-    }
-    return Status::OK();
-  }
-};
-
-class ListConverter : public TypedConverter<ListBuilder> {
- public:
-  Status Init(const std::shared_ptr<ArrayBuilder>& builder) override;
-
-  Status AppendData(PyObject* seq) override {
-    Py_ssize_t size = PySequence_Size(seq);
-    for (int64_t i = 0; i < size; ++i) {
-      OwnedRef item(PySequence_GetItem(seq, i));
-      if (item.obj() == Py_None) {
-        RETURN_NOT_OK(typed_builder_->AppendNull());
-      } else {
-        typed_builder_->Append();
-        RETURN_NOT_OK(value_converter_->AppendData(item.obj()));
-      }
-    }
-    return Status::OK();
-  }
-
- protected:
-  std::shared_ptr<SeqConverter> value_converter_;
-};
-
-// Dynamic constructor for sequence converters
-std::shared_ptr<SeqConverter> GetConverter(const std::shared_ptr<DataType>& type) {
-  switch (type->type) {
-    case Type::BOOL:
-      return std::make_shared<BoolConverter>();
-    case Type::INT64:
-      return std::make_shared<Int64Converter>();
-    case Type::DATE64:
-      return std::make_shared<DateConverter>();
-    case Type::TIMESTAMP:
-      return std::make_shared<TimestampConverter>();
-    case Type::DOUBLE:
-      return std::make_shared<DoubleConverter>();
-    case Type::BINARY:
-      return std::make_shared<BytesConverter>();
-    case Type::STRING:
-      return std::make_shared<UTF8Converter>();
-    case Type::LIST:
-      return std::make_shared<ListConverter>();
-    case Type::STRUCT:
-    default:
-      return nullptr;
-      break;
-  }
-}
-
-Status ListConverter::Init(const std::shared_ptr<ArrayBuilder>& builder) {
-  builder_ = builder;
-  typed_builder_ = static_cast<ListBuilder*>(builder.get());
-
-  value_converter_ =
-      GetConverter(static_cast<ListType*>(builder->type().get())->value_type());
-  if (value_converter_ == nullptr) {
-    return Status::NotImplemented("value type not implemented");
-  }
-
-  value_converter_->Init(typed_builder_->value_builder());
-  return Status::OK();
-}
-
-Status AppendPySequence(PyObject* obj, const std::shared_ptr<DataType>& type,
-    const std::shared_ptr<ArrayBuilder>& builder) {
-  std::shared_ptr<SeqConverter> converter = GetConverter(type);
-  if (converter == nullptr) {
-    std::stringstream ss;
-    ss << "No type converter implemented for " << type->ToString();
-    return Status::NotImplemented(ss.str());
-  }
-  converter->Init(builder);
-
-  return converter->AppendData(obj);
-}
-
-Status ConvertPySequence(PyObject* obj, MemoryPool* pool, std::shared_ptr<Array>* out) {
-  std::shared_ptr<DataType> type;
-  int64_t size;
-  PyDateTime_IMPORT;
-  RETURN_NOT_OK(InferArrowType(obj, &size, &type));
-
-  // Handle NA / NullType case
-  if (type->type == Type::NA) {
-    out->reset(new NullArray(size));
-    return Status::OK();
-  }
-
-  // Give the sequence converter an array builder
-  std::shared_ptr<ArrayBuilder> builder;
-  RETURN_NOT_OK(MakeBuilder(pool, type, &builder));
-  RETURN_NOT_OK(AppendPySequence(obj, type, builder));
-
-  return builder->Finish(out);
-}
-
-}  // namespace py
-}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/adapters/builtin.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/builtin.h b/python/src/pyarrow/adapters/builtin.h
deleted file mode 100644
index 2d45e67..0000000
--- a/python/src/pyarrow/adapters/builtin.h
+++ /dev/null
@@ -1,54 +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.
-
-// Functions for converting between CPython built-in data structures and Arrow
-// data structures
-
-#ifndef PYARROW_ADAPTERS_BUILTIN_H
-#define PYARROW_ADAPTERS_BUILTIN_H
-
-#include <Python.h>
-
-#include <memory>
-
-#include <arrow/type.h>
-
-#include "arrow/util/visibility.h"
-
-#include "pyarrow/common.h"
-
-namespace arrow {
-
-class Array;
-class Status;
-
-namespace py {
-
-ARROW_EXPORT arrow::Status InferArrowType(
-    PyObject* obj, int64_t* size, std::shared_ptr<arrow::DataType>* out_type);
-
-ARROW_EXPORT arrow::Status AppendPySequence(PyObject* obj,
-    const std::shared_ptr<arrow::DataType>& type,
-    const std::shared_ptr<arrow::ArrayBuilder>& builder);
-
-ARROW_EXPORT
-Status ConvertPySequence(PyObject* obj, MemoryPool* pool, std::shared_ptr<Array>* out);
-
-}  // namespace py
-}  // namespace arrow
-
-#endif  // PYARROW_ADAPTERS_BUILTIN_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/adapters/pandas-test.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/pandas-test.cc b/python/src/pyarrow/adapters/pandas-test.cc
deleted file mode 100644
index e694e79..0000000
--- a/python/src/pyarrow/adapters/pandas-test.cc
+++ /dev/null
@@ -1,64 +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/schema.h"
-#include "arrow/table.h"
-#include "arrow/test-util.h"
-#include "arrow/type.h"
-#include "pyarrow/adapters/pandas.h"
-
-namespace arrow {
-namespace py {
-
-TEST(PandasConversionTest, TestObjectBlockWriteFails) {
-  StringBuilder builder;
-  const char value[] = {'\xf1', '\0'};
-
-  for (int i = 0; i < 1000; ++i) {
-    builder.Append(value, 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


[4/5] arrow git commit: ARROW-341: [Python] Move pyarrow's C++ code to the main C++ source tree, install libarrow_python and headers

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/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
new file mode 100644
index 0000000..f2c2415
--- /dev/null
+++ b/cpp/src/arrow/python/pandas_convert.cc
@@ -0,0 +1,1936 @@
+// 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.
+
+// Functions for pandas conversion via NumPy
+
+#include <Python.h>
+
+#include "arrow/python/numpy_interop.h"
+#include "arrow/python/pandas_convert.h"
+
+#include <algorithm>
+#include <atomic>
+#include <cmath>
+#include <cstdint>
+#include <memory>
+#include <mutex>
+#include <sstream>
+#include <string>
+#include <thread>
+#include <unordered_map>
+#include <vector>
+
+#include "arrow/array.h"
+#include "arrow/column.h"
+#include "arrow/loader.h"
+#include "arrow/python/builtin_convert.h"
+#include "arrow/python/common.h"
+#include "arrow/python/config.h"
+#include "arrow/python/type_traits.h"
+#include "arrow/python/util/datetime.h"
+#include "arrow/status.h"
+#include "arrow/table.h"
+#include "arrow/type_fwd.h"
+#include "arrow/type_traits.h"
+#include "arrow/util/bit-util.h"
+#include "arrow/util/macros.h"
+
+namespace arrow {
+namespace py {
+
+// ----------------------------------------------------------------------
+// Utility code
+
+int cast_npy_type_compat(int type_num) {
+// Both LONGLONG and INT64 can be observed in the wild, which is buggy. We set
+// U/LONGLONG to U/INT64 so things work properly.
+
+#if (NPY_INT64 == NPY_LONGLONG) && (NPY_SIZEOF_LONGLONG == 8)
+  if (type_num == NPY_LONGLONG) { type_num = NPY_INT64; }
+  if (type_num == NPY_ULONGLONG) { type_num = NPY_UINT64; }
+#endif
+
+  return type_num;
+}
+
+static inline bool PyObject_is_null(const PyObject* obj) {
+  return obj == Py_None || obj == numpy_nan;
+}
+
+static inline bool PyObject_is_string(const PyObject* obj) {
+#if PY_MAJOR_VERSION >= 3
+  return PyUnicode_Check(obj) || PyBytes_Check(obj);
+#else
+  return PyString_Check(obj) || PyUnicode_Check(obj);
+#endif
+}
+
+template <int TYPE>
+static int64_t ValuesToBitmap(const void* data, int64_t length, uint8_t* bitmap) {
+  typedef npy_traits<TYPE> traits;
+  typedef typename traits::value_type T;
+
+  int64_t null_count = 0;
+  const T* values = reinterpret_cast<const T*>(data);
+
+  // TODO(wesm): striding
+  for (int i = 0; i < length; ++i) {
+    if (traits::isnull(values[i])) {
+      ++null_count;
+    } else {
+      BitUtil::SetBit(bitmap, i);
+    }
+  }
+
+  return null_count;
+}
+
+// Returns null count
+static int64_t MaskToBitmap(PyArrayObject* mask, int64_t length, uint8_t* bitmap) {
+  int64_t null_count = 0;
+  const uint8_t* mask_values = static_cast<const uint8_t*>(PyArray_DATA(mask));
+  // TODO(wesm): strided null mask
+  for (int i = 0; i < length; ++i) {
+    if (mask_values[i]) {
+      ++null_count;
+    } else {
+      BitUtil::SetBit(bitmap, i);
+    }
+  }
+  return null_count;
+}
+
+template <int TYPE>
+static int64_t ValuesToValidBytes(
+    const void* data, int64_t length, uint8_t* valid_bytes) {
+  typedef npy_traits<TYPE> traits;
+  typedef typename traits::value_type T;
+
+  int64_t null_count = 0;
+  const T* values = reinterpret_cast<const T*>(data);
+
+  // TODO(wesm): striding
+  for (int i = 0; i < length; ++i) {
+    valid_bytes[i] = !traits::isnull(values[i]);
+    if (traits::isnull(values[i])) null_count++;
+  }
+
+  return null_count;
+}
+
+Status CheckFlatNumpyArray(PyArrayObject* numpy_array, int np_type) {
+  if (PyArray_NDIM(numpy_array) != 1) {
+    return Status::Invalid("only handle 1-dimensional arrays");
+  }
+
+  if (PyArray_DESCR(numpy_array)->type_num != np_type) {
+    return Status::Invalid("can only handle exact conversions");
+  }
+
+  npy_intp* astrides = PyArray_STRIDES(numpy_array);
+  if (astrides[0] != PyArray_DESCR(numpy_array)->elsize) {
+    return Status::Invalid("No support for strided arrays in lists yet");
+  }
+  return Status::OK();
+}
+
+Status AppendObjectStrings(StringBuilder& string_builder, PyObject** objects,
+    int64_t objects_length, bool* have_bytes) {
+  PyObject* obj;
+
+  for (int64_t i = 0; i < objects_length; ++i) {
+    obj = objects[i];
+    if (PyUnicode_Check(obj)) {
+      obj = PyUnicode_AsUTF8String(obj);
+      if (obj == NULL) {
+        PyErr_Clear();
+        return Status::TypeError("failed converting unicode to UTF8");
+      }
+      const int64_t length = PyBytes_GET_SIZE(obj);
+      Status s = string_builder.Append(PyBytes_AS_STRING(obj), length);
+      Py_DECREF(obj);
+      if (!s.ok()) { return s; }
+    } else if (PyBytes_Check(obj)) {
+      *have_bytes = true;
+      const int64_t length = PyBytes_GET_SIZE(obj);
+      RETURN_NOT_OK(string_builder.Append(PyBytes_AS_STRING(obj), length));
+    } else {
+      string_builder.AppendNull();
+    }
+  }
+
+  return Status::OK();
+}
+
+template <typename T>
+struct WrapBytes {};
+
+template <>
+struct WrapBytes<StringArray> {
+  static inline PyObject* Wrap(const uint8_t* data, int64_t length) {
+    return PyUnicode_FromStringAndSize(reinterpret_cast<const char*>(data), length);
+  }
+};
+
+template <>
+struct WrapBytes<BinaryArray> {
+  static inline PyObject* Wrap(const uint8_t* data, int64_t length) {
+    return PyBytes_FromStringAndSize(reinterpret_cast<const char*>(data), length);
+  }
+};
+
+static inline bool ListTypeSupported(const Type::type type_id) {
+  switch (type_id) {
+    case Type::UINT8:
+    case Type::INT8:
+    case Type::UINT16:
+    case Type::INT16:
+    case Type::UINT32:
+    case Type::INT32:
+    case Type::INT64:
+    case Type::UINT64:
+    case Type::FLOAT:
+    case Type::DOUBLE:
+    case Type::STRING:
+    case Type::TIMESTAMP:
+      // The above types are all supported.
+      return true;
+    default:
+      break;
+  }
+  return false;
+}
+
+// ----------------------------------------------------------------------
+// Conversion from NumPy-in-Pandas to Arrow
+
+class PandasConverter : public TypeVisitor {
+ public:
+  PandasConverter(
+      MemoryPool* pool, PyObject* ao, PyObject* mo, const std::shared_ptr<DataType>& type)
+      : pool_(pool),
+        type_(type),
+        arr_(reinterpret_cast<PyArrayObject*>(ao)),
+        mask_(nullptr) {
+    if (mo != nullptr && mo != Py_None) { mask_ = reinterpret_cast<PyArrayObject*>(mo); }
+    length_ = PyArray_SIZE(arr_);
+  }
+
+  bool is_strided() const {
+    npy_intp* astrides = PyArray_STRIDES(arr_);
+    return astrides[0] != PyArray_DESCR(arr_)->elsize;
+  }
+
+  Status InitNullBitmap() {
+    int null_bytes = BitUtil::BytesForBits(length_);
+
+    null_bitmap_ = std::make_shared<PoolBuffer>(pool_);
+    RETURN_NOT_OK(null_bitmap_->Resize(null_bytes));
+
+    null_bitmap_data_ = null_bitmap_->mutable_data();
+    memset(null_bitmap_data_, 0, null_bytes);
+
+    return Status::OK();
+  }
+
+  // ----------------------------------------------------------------------
+  // Traditional visitor conversion for non-object arrays
+
+  template <typename ArrowType>
+  Status ConvertData(std::shared_ptr<Buffer>* data);
+
+  template <typename ArrowType>
+  Status VisitNative() {
+    using traits = arrow_traits<ArrowType::type_id>;
+
+    if (mask_ != nullptr || traits::supports_nulls) { RETURN_NOT_OK(InitNullBitmap()); }
+
+    std::shared_ptr<Buffer> data;
+    RETURN_NOT_OK(ConvertData<ArrowType>(&data));
+
+    int64_t null_count = 0;
+    if (mask_ != nullptr) {
+      null_count = MaskToBitmap(mask_, length_, null_bitmap_data_);
+    } else if (traits::supports_nulls) {
+      // TODO(wesm): this presumes the NumPy C type and arrow C type are the
+      // same
+      null_count = ValuesToBitmap<traits::npy_type>(
+          PyArray_DATA(arr_), length_, null_bitmap_data_);
+    }
+
+    std::vector<FieldMetadata> fields(1);
+    fields[0].length = length_;
+    fields[0].null_count = null_count;
+    fields[0].offset = 0;
+
+    return LoadArray(type_, fields, {null_bitmap_, data}, &out_);
+  }
+
+#define VISIT_NATIVE(TYPE) \
+  Status Visit(const TYPE& type) override { return VisitNative<TYPE>(); }
+
+  VISIT_NATIVE(BooleanType);
+  VISIT_NATIVE(Int8Type);
+  VISIT_NATIVE(Int16Type);
+  VISIT_NATIVE(Int32Type);
+  VISIT_NATIVE(Int64Type);
+  VISIT_NATIVE(UInt8Type);
+  VISIT_NATIVE(UInt16Type);
+  VISIT_NATIVE(UInt32Type);
+  VISIT_NATIVE(UInt64Type);
+  VISIT_NATIVE(FloatType);
+  VISIT_NATIVE(DoubleType);
+  VISIT_NATIVE(TimestampType);
+
+#undef VISIT_NATIVE
+
+  Status Convert(std::shared_ptr<Array>* out) {
+    if (PyArray_NDIM(arr_) != 1) {
+      return Status::Invalid("only handle 1-dimensional arrays");
+    }
+    // TODO(wesm): strided arrays
+    if (is_strided()) { return Status::Invalid("no support for strided data yet"); }
+
+    if (type_ == nullptr) { return Status::Invalid("Must pass data type"); }
+
+    // Visit the type to perform conversion
+    RETURN_NOT_OK(type_->Accept(this));
+
+    *out = out_;
+    return Status::OK();
+  }
+
+  // ----------------------------------------------------------------------
+  // Conversion logic for various object dtype arrays
+
+  template <int ITEM_TYPE, typename ArrowType>
+  Status ConvertTypedLists(
+      const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out);
+
+  Status ConvertObjectStrings(std::shared_ptr<Array>* out);
+  Status ConvertBooleans(std::shared_ptr<Array>* out);
+  Status ConvertDates(std::shared_ptr<Array>* out);
+  Status ConvertLists(const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out);
+  Status ConvertObjects(std::shared_ptr<Array>* out);
+
+ protected:
+  MemoryPool* pool_;
+  std::shared_ptr<DataType> type_;
+  PyArrayObject* arr_;
+  PyArrayObject* mask_;
+  int64_t length_;
+
+  // Used in visitor pattern
+  std::shared_ptr<Array> out_;
+
+  std::shared_ptr<ResizableBuffer> null_bitmap_;
+  uint8_t* null_bitmap_data_;
+};
+
+template <typename ArrowType>
+inline Status PandasConverter::ConvertData(std::shared_ptr<Buffer>* data) {
+  using traits = arrow_traits<ArrowType::type_id>;
+
+  // Handle LONGLONG->INT64 and other fun things
+  int type_num_compat = cast_npy_type_compat(PyArray_DESCR(arr_)->type_num);
+
+  if (traits::npy_type != type_num_compat) {
+    return Status::NotImplemented("NumPy type casts not yet implemented");
+  }
+
+  *data = std::make_shared<NumPyBuffer>(arr_);
+  return Status::OK();
+}
+
+template <>
+inline Status PandasConverter::ConvertData<BooleanType>(std::shared_ptr<Buffer>* data) {
+  int nbytes = BitUtil::BytesForBits(length_);
+  auto buffer = std::make_shared<PoolBuffer>(pool_);
+  RETURN_NOT_OK(buffer->Resize(nbytes));
+
+  const uint8_t* values = reinterpret_cast<const uint8_t*>(PyArray_DATA(arr_));
+
+  uint8_t* bitmap = buffer->mutable_data();
+
+  memset(bitmap, 0, nbytes);
+  for (int i = 0; i < length_; ++i) {
+    if (values[i] > 0) { BitUtil::SetBit(bitmap, i); }
+  }
+
+  *data = buffer;
+  return Status::OK();
+}
+
+Status PandasConverter::ConvertDates(std::shared_ptr<Array>* out) {
+  PyAcquireGIL lock;
+
+  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
+  Date64Builder date_builder(pool_);
+  RETURN_NOT_OK(date_builder.Resize(length_));
+
+  Status s;
+  PyObject* obj;
+  for (int64_t i = 0; i < length_; ++i) {
+    obj = objects[i];
+    if (PyDate_CheckExact(obj)) {
+      PyDateTime_Date* pydate = reinterpret_cast<PyDateTime_Date*>(obj);
+      date_builder.Append(PyDate_to_ms(pydate));
+    } else {
+      date_builder.AppendNull();
+    }
+  }
+  return date_builder.Finish(out);
+}
+
+Status PandasConverter::ConvertObjectStrings(std::shared_ptr<Array>* out) {
+  PyAcquireGIL lock;
+
+  // The output type at this point is inconclusive because there may be bytes
+  // and unicode mixed in the object array
+
+  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
+  StringBuilder string_builder(pool_);
+  RETURN_NOT_OK(string_builder.Resize(length_));
+
+  Status s;
+  bool have_bytes = false;
+  RETURN_NOT_OK(AppendObjectStrings(string_builder, objects, length_, &have_bytes));
+  RETURN_NOT_OK(string_builder.Finish(out));
+
+  if (have_bytes) {
+    const auto& arr = static_cast<const StringArray&>(*out->get());
+    *out = std::make_shared<BinaryArray>(arr.length(), arr.value_offsets(), arr.data(),
+        arr.null_bitmap(), arr.null_count());
+  }
+  return Status::OK();
+}
+
+Status PandasConverter::ConvertBooleans(std::shared_ptr<Array>* out) {
+  PyAcquireGIL lock;
+
+  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
+
+  int nbytes = BitUtil::BytesForBits(length_);
+  auto data = std::make_shared<PoolBuffer>(pool_);
+  RETURN_NOT_OK(data->Resize(nbytes));
+  uint8_t* bitmap = data->mutable_data();
+  memset(bitmap, 0, nbytes);
+
+  int64_t null_count = 0;
+  for (int64_t i = 0; i < length_; ++i) {
+    if (objects[i] == Py_True) {
+      BitUtil::SetBit(bitmap, i);
+      BitUtil::SetBit(null_bitmap_data_, i);
+    } else if (objects[i] != Py_False) {
+      ++null_count;
+    } else {
+      BitUtil::SetBit(null_bitmap_data_, i);
+    }
+  }
+
+  *out = std::make_shared<BooleanArray>(length_, data, null_bitmap_, null_count);
+
+  return Status::OK();
+}
+
+Status PandasConverter::ConvertObjects(std::shared_ptr<Array>* out) {
+  // Python object arrays are annoying, since we could have one of:
+  //
+  // * Strings
+  // * Booleans with nulls
+  // * Mixed type (not supported at the moment by arrow format)
+  //
+  // Additionally, nulls may be encoded either as np.nan or None. So we have to
+  // do some type inference and conversion
+
+  RETURN_NOT_OK(InitNullBitmap());
+
+  // TODO: mask not supported here
+  if (mask_ != nullptr) {
+    return Status::NotImplemented("mask not supported in object conversions yet");
+  }
+
+  const PyObject** objects;
+  {
+    PyAcquireGIL lock;
+    objects = reinterpret_cast<const PyObject**>(PyArray_DATA(arr_));
+    PyDateTime_IMPORT;
+  }
+
+  if (type_) {
+    switch (type_->type) {
+      case Type::STRING:
+        return ConvertObjectStrings(out);
+      case Type::BOOL:
+        return ConvertBooleans(out);
+      case Type::DATE64:
+        return ConvertDates(out);
+      case Type::LIST: {
+        const auto& list_field = static_cast<const ListType&>(*type_);
+        return ConvertLists(list_field.value_field()->type, out);
+      }
+      default:
+        return Status::TypeError("No known conversion to Arrow type");
+    }
+  } else {
+    for (int64_t i = 0; i < length_; ++i) {
+      if (PyObject_is_null(objects[i])) {
+        continue;
+      } else if (PyObject_is_string(objects[i])) {
+        return ConvertObjectStrings(out);
+      } else if (PyBool_Check(objects[i])) {
+        return ConvertBooleans(out);
+      } else if (PyDate_CheckExact(objects[i])) {
+        return ConvertDates(out);
+      } else {
+        return Status::TypeError("unhandled python type");
+      }
+    }
+  }
+
+  return Status::TypeError("Unable to infer type of object array, were all null");
+}
+
+template <int ITEM_TYPE, typename ArrowType>
+inline Status PandasConverter::ConvertTypedLists(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
+  typedef npy_traits<ITEM_TYPE> traits;
+  typedef typename traits::value_type T;
+  typedef typename traits::BuilderClass BuilderT;
+
+  PyAcquireGIL lock;
+
+  auto value_builder = std::make_shared<BuilderT>(pool_, type);
+  ListBuilder list_builder(pool_, value_builder);
+  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
+  for (int64_t i = 0; i < length_; ++i) {
+    if (PyObject_is_null(objects[i])) {
+      RETURN_NOT_OK(list_builder.AppendNull());
+    } else if (PyArray_Check(objects[i])) {
+      auto numpy_array = reinterpret_cast<PyArrayObject*>(objects[i]);
+      RETURN_NOT_OK(list_builder.Append(true));
+
+      // TODO(uwe): Support more complex numpy array structures
+      RETURN_NOT_OK(CheckFlatNumpyArray(numpy_array, ITEM_TYPE));
+
+      int64_t size = PyArray_DIM(numpy_array, 0);
+      auto data = reinterpret_cast<const T*>(PyArray_DATA(numpy_array));
+      if (traits::supports_nulls) {
+        null_bitmap_->Resize(size, false);
+        // TODO(uwe): A bitmap would be more space-efficient but the Builder API doesn't
+        // currently support this.
+        // ValuesToBitmap<ITEM_TYPE>(data, size, null_bitmap_->mutable_data());
+        ValuesToValidBytes<ITEM_TYPE>(data, size, null_bitmap_->mutable_data());
+        RETURN_NOT_OK(value_builder->Append(data, size, null_bitmap_->data()));
+      } else {
+        RETURN_NOT_OK(value_builder->Append(data, size));
+      }
+
+    } else if (PyList_Check(objects[i])) {
+      int64_t size;
+      std::shared_ptr<DataType> inferred_type;
+      RETURN_NOT_OK(list_builder.Append(true));
+      RETURN_NOT_OK(InferArrowType(objects[i], &size, &inferred_type));
+      if (inferred_type->type != type->type) {
+        std::stringstream ss;
+        ss << inferred_type->ToString() << " cannot be converted to " << type->ToString();
+        return Status::TypeError(ss.str());
+      }
+      RETURN_NOT_OK(AppendPySequence(objects[i], type, value_builder));
+    } else {
+      return Status::TypeError("Unsupported Python type for list items");
+    }
+  }
+  return list_builder.Finish(out);
+}
+
+template <>
+inline Status PandasConverter::ConvertTypedLists<NPY_OBJECT, StringType>(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
+  PyAcquireGIL lock;
+  // TODO: If there are bytes involed, convert to Binary representation
+  bool have_bytes = false;
+
+  auto value_builder = std::make_shared<StringBuilder>(pool_);
+  ListBuilder list_builder(pool_, value_builder);
+  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
+  for (int64_t i = 0; i < length_; ++i) {
+    if (PyObject_is_null(objects[i])) {
+      RETURN_NOT_OK(list_builder.AppendNull());
+    } else if (PyArray_Check(objects[i])) {
+      auto numpy_array = reinterpret_cast<PyArrayObject*>(objects[i]);
+      RETURN_NOT_OK(list_builder.Append(true));
+
+      // TODO(uwe): Support more complex numpy array structures
+      RETURN_NOT_OK(CheckFlatNumpyArray(numpy_array, NPY_OBJECT));
+
+      int64_t size = PyArray_DIM(numpy_array, 0);
+      auto data = reinterpret_cast<PyObject**>(PyArray_DATA(numpy_array));
+      RETURN_NOT_OK(AppendObjectStrings(*value_builder.get(), data, size, &have_bytes));
+    } else if (PyList_Check(objects[i])) {
+      int64_t size;
+      std::shared_ptr<DataType> inferred_type;
+      RETURN_NOT_OK(list_builder.Append(true));
+      RETURN_NOT_OK(InferArrowType(objects[i], &size, &inferred_type));
+      if (inferred_type->type != Type::STRING) {
+        std::stringstream ss;
+        ss << inferred_type->ToString() << " cannot be converted to STRING.";
+        return Status::TypeError(ss.str());
+      }
+      RETURN_NOT_OK(AppendPySequence(objects[i], inferred_type, value_builder));
+    } else {
+      return Status::TypeError("Unsupported Python type for list items");
+    }
+  }
+  return list_builder.Finish(out);
+}
+
+#define LIST_CASE(TYPE, NUMPY_TYPE, ArrowType)                  \
+  case Type::TYPE: {                                            \
+    return ConvertTypedLists<NUMPY_TYPE, ArrowType>(type, out); \
+  }
+
+Status PandasConverter::ConvertLists(
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
+  switch (type->type) {
+    LIST_CASE(UINT8, NPY_UINT8, UInt8Type)
+    LIST_CASE(INT8, NPY_INT8, Int8Type)
+    LIST_CASE(UINT16, NPY_UINT16, UInt16Type)
+    LIST_CASE(INT16, NPY_INT16, Int16Type)
+    LIST_CASE(UINT32, NPY_UINT32, UInt32Type)
+    LIST_CASE(INT32, NPY_INT32, Int32Type)
+    LIST_CASE(UINT64, NPY_UINT64, UInt64Type)
+    LIST_CASE(INT64, NPY_INT64, Int64Type)
+    LIST_CASE(TIMESTAMP, NPY_DATETIME, TimestampType)
+    LIST_CASE(FLOAT, NPY_FLOAT, FloatType)
+    LIST_CASE(DOUBLE, NPY_DOUBLE, DoubleType)
+    LIST_CASE(STRING, NPY_OBJECT, StringType)
+    default:
+      return Status::TypeError("Unknown list item type");
+  }
+
+  return Status::TypeError("Unknown list type");
+}
+
+Status PandasToArrow(MemoryPool* pool, PyObject* ao, PyObject* mo,
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
+  PandasConverter converter(pool, ao, mo, type);
+  return converter.Convert(out);
+}
+
+Status PandasObjectsToArrow(MemoryPool* pool, PyObject* ao, PyObject* mo,
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
+  PandasConverter converter(pool, ao, mo, type);
+  return converter.ConvertObjects(out);
+}
+
+Status PandasDtypeToArrow(PyObject* dtype, std::shared_ptr<DataType>* out) {
+  PyArray_Descr* descr = reinterpret_cast<PyArray_Descr*>(dtype);
+
+  int type_num = cast_npy_type_compat(descr->type_num);
+
+#define TO_ARROW_TYPE_CASE(NPY_NAME, FACTORY) \
+  case NPY_##NPY_NAME:                        \
+    *out = FACTORY();                         \
+    break;
+
+  switch (type_num) {
+    TO_ARROW_TYPE_CASE(BOOL, boolean);
+    TO_ARROW_TYPE_CASE(INT8, int8);
+    TO_ARROW_TYPE_CASE(INT16, int16);
+    TO_ARROW_TYPE_CASE(INT32, int32);
+    TO_ARROW_TYPE_CASE(INT64, int64);
+#if (NPY_INT64 != NPY_LONGLONG)
+    TO_ARROW_TYPE_CASE(LONGLONG, int64);
+#endif
+    TO_ARROW_TYPE_CASE(UINT8, uint8);
+    TO_ARROW_TYPE_CASE(UINT16, uint16);
+    TO_ARROW_TYPE_CASE(UINT32, uint32);
+    TO_ARROW_TYPE_CASE(UINT64, uint64);
+#if (NPY_UINT64 != NPY_ULONGLONG)
+    TO_ARROW_CASE(ULONGLONG);
+#endif
+    TO_ARROW_TYPE_CASE(FLOAT32, float32);
+    TO_ARROW_TYPE_CASE(FLOAT64, float64);
+    case NPY_DATETIME: {
+      auto date_dtype =
+          reinterpret_cast<PyArray_DatetimeDTypeMetaData*>(descr->c_metadata);
+      TimeUnit unit;
+      switch (date_dtype->meta.base) {
+        case NPY_FR_s:
+          unit = TimeUnit::SECOND;
+          break;
+        case NPY_FR_ms:
+          unit = TimeUnit::MILLI;
+          break;
+        case NPY_FR_us:
+          unit = TimeUnit::MICRO;
+          break;
+        case NPY_FR_ns:
+          unit = TimeUnit::NANO;
+          break;
+        default:
+          return Status::NotImplemented("Unsupported datetime64 time unit");
+      }
+      *out = timestamp(unit);
+    } break;
+    default: {
+      std::stringstream ss;
+      ss << "Unsupported numpy type " << descr->type_num << std::endl;
+      return Status::NotImplemented(ss.str());
+    }
+  }
+
+#undef TO_ARROW_TYPE_CASE
+
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// pandas 0.x DataFrame conversion internals
+
+inline void set_numpy_metadata(int type, DataType* datatype, PyArrayObject* out) {
+  if (type == NPY_DATETIME) {
+    PyArray_Descr* descr = PyArray_DESCR(out);
+    auto date_dtype = reinterpret_cast<PyArray_DatetimeDTypeMetaData*>(descr->c_metadata);
+    if (datatype->type == Type::TIMESTAMP) {
+      auto timestamp_type = static_cast<TimestampType*>(datatype);
+
+      switch (timestamp_type->unit) {
+        case TimestampType::Unit::SECOND:
+          date_dtype->meta.base = NPY_FR_s;
+          break;
+        case TimestampType::Unit::MILLI:
+          date_dtype->meta.base = NPY_FR_ms;
+          break;
+        case TimestampType::Unit::MICRO:
+          date_dtype->meta.base = NPY_FR_us;
+          break;
+        case TimestampType::Unit::NANO:
+          date_dtype->meta.base = NPY_FR_ns;
+          break;
+      }
+    } else {
+      // datatype->type == Type::DATE64
+      date_dtype->meta.base = NPY_FR_D;
+    }
+  }
+}
+
+class PandasBlock {
+ public:
+  enum type {
+    OBJECT,
+    UINT8,
+    INT8,
+    UINT16,
+    INT16,
+    UINT32,
+    INT32,
+    UINT64,
+    INT64,
+    FLOAT,
+    DOUBLE,
+    BOOL,
+    DATETIME,
+    DATETIME_WITH_TZ,
+    CATEGORICAL
+  };
+
+  PandasBlock(int64_t num_rows, int num_columns)
+      : num_rows_(num_rows), num_columns_(num_columns) {}
+  virtual ~PandasBlock() {}
+
+  virtual Status Allocate() = 0;
+  virtual Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
+      int64_t rel_placement) = 0;
+
+  PyObject* block_arr() const { return block_arr_.obj(); }
+
+  virtual Status GetPyResult(PyObject** output) {
+    PyObject* result = PyDict_New();
+    RETURN_IF_PYERROR();
+
+    PyDict_SetItemString(result, "block", block_arr_.obj());
+    PyDict_SetItemString(result, "placement", placement_arr_.obj());
+
+    *output = result;
+
+    return Status::OK();
+  }
+
+ protected:
+  Status AllocateNDArray(int npy_type, int ndim = 2) {
+    PyAcquireGIL lock;
+
+    PyObject* block_arr;
+    if (ndim == 2) {
+      npy_intp block_dims[2] = {num_columns_, num_rows_};
+      block_arr = PyArray_SimpleNew(2, block_dims, npy_type);
+    } else {
+      npy_intp block_dims[1] = {num_rows_};
+      block_arr = PyArray_SimpleNew(1, block_dims, npy_type);
+    }
+
+    if (block_arr == NULL) {
+      // TODO(wesm): propagating Python exception
+      return Status::OK();
+    }
+
+    npy_intp placement_dims[1] = {num_columns_};
+    PyObject* placement_arr = PyArray_SimpleNew(1, placement_dims, NPY_INT64);
+    if (placement_arr == NULL) {
+      // TODO(wesm): propagating Python exception
+      return Status::OK();
+    }
+
+    block_arr_.reset(block_arr);
+    placement_arr_.reset(placement_arr);
+
+    block_data_ = reinterpret_cast<uint8_t*>(
+        PyArray_DATA(reinterpret_cast<PyArrayObject*>(block_arr)));
+
+    placement_data_ = reinterpret_cast<int64_t*>(
+        PyArray_DATA(reinterpret_cast<PyArrayObject*>(placement_arr)));
+
+    return Status::OK();
+  }
+
+  int64_t num_rows_;
+  int num_columns_;
+
+  OwnedRef block_arr_;
+  uint8_t* block_data_;
+
+  // ndarray<int32>
+  OwnedRef placement_arr_;
+  int64_t* placement_data_;
+
+ private:
+  DISALLOW_COPY_AND_ASSIGN(PandasBlock);
+};
+
+template <typename T>
+inline void ConvertIntegerWithNulls(const ChunkedArray& data, double* out_values) {
+  for (int c = 0; c < data.num_chunks(); c++) {
+    const std::shared_ptr<Array> arr = data.chunk(c);
+    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
+    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
+    // Upcast to double, set NaN as appropriate
+
+    for (int i = 0; i < arr->length(); ++i) {
+      *out_values++ = prim_arr->IsNull(i) ? NAN : in_values[i];
+    }
+  }
+}
+
+template <typename T>
+inline void ConvertIntegerNoNullsSameType(const ChunkedArray& data, T* out_values) {
+  for (int c = 0; c < data.num_chunks(); c++) {
+    const std::shared_ptr<Array> arr = data.chunk(c);
+    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
+    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
+    memcpy(out_values, in_values, sizeof(T) * arr->length());
+    out_values += arr->length();
+  }
+}
+
+template <typename InType, typename OutType>
+inline void ConvertIntegerNoNullsCast(const ChunkedArray& data, OutType* out_values) {
+  for (int c = 0; c < data.num_chunks(); c++) {
+    const std::shared_ptr<Array> arr = data.chunk(c);
+    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
+    auto in_values = reinterpret_cast<const InType*>(prim_arr->data()->data());
+    for (int64_t i = 0; i < arr->length(); ++i) {
+      *out_values = in_values[i];
+    }
+  }
+}
+
+static Status ConvertBooleanWithNulls(const ChunkedArray& data, PyObject** out_values) {
+  PyAcquireGIL lock;
+  for (int c = 0; c < data.num_chunks(); c++) {
+    const std::shared_ptr<Array> arr = data.chunk(c);
+    auto bool_arr = static_cast<BooleanArray*>(arr.get());
+
+    for (int64_t i = 0; i < arr->length(); ++i) {
+      if (bool_arr->IsNull(i)) {
+        Py_INCREF(Py_None);
+        *out_values++ = Py_None;
+      } else if (bool_arr->Value(i)) {
+        // True
+        Py_INCREF(Py_True);
+        *out_values++ = Py_True;
+      } else {
+        // False
+        Py_INCREF(Py_False);
+        *out_values++ = Py_False;
+      }
+    }
+  }
+  return Status::OK();
+}
+
+static void ConvertBooleanNoNulls(const ChunkedArray& data, uint8_t* out_values) {
+  for (int c = 0; c < data.num_chunks(); c++) {
+    const std::shared_ptr<Array> arr = data.chunk(c);
+    auto bool_arr = static_cast<BooleanArray*>(arr.get());
+    for (int64_t i = 0; i < arr->length(); ++i) {
+      *out_values++ = static_cast<uint8_t>(bool_arr->Value(i));
+    }
+  }
+}
+
+template <typename ArrayType>
+inline Status ConvertBinaryLike(const ChunkedArray& data, PyObject** out_values) {
+  PyAcquireGIL lock;
+  for (int c = 0; c < data.num_chunks(); c++) {
+    auto arr = static_cast<ArrayType*>(data.chunk(c).get());
+
+    const uint8_t* data_ptr;
+    int32_t length;
+    const bool has_nulls = data.null_count() > 0;
+    for (int64_t i = 0; i < arr->length(); ++i) {
+      if (has_nulls && arr->IsNull(i)) {
+        Py_INCREF(Py_None);
+        *out_values = Py_None;
+      } else {
+        data_ptr = arr->GetValue(i, &length);
+        *out_values = WrapBytes<ArrayType>::Wrap(data_ptr, length);
+        if (*out_values == nullptr) {
+          PyErr_Clear();
+          std::stringstream ss;
+          ss << "Wrapping "
+             << std::string(reinterpret_cast<const char*>(data_ptr), length) << " failed";
+          return Status::UnknownError(ss.str());
+        }
+      }
+      ++out_values;
+    }
+  }
+  return Status::OK();
+}
+
+template <typename ArrowType>
+inline Status ConvertListsLike(
+    const std::shared_ptr<Column>& col, PyObject** out_values) {
+  const ChunkedArray& data = *col->data().get();
+  auto list_type = std::static_pointer_cast<ListType>(col->type());
+
+  // Get column of underlying value arrays
+  std::vector<std::shared_ptr<Array>> value_arrays;
+  for (int c = 0; c < data.num_chunks(); c++) {
+    auto arr = std::static_pointer_cast<ListArray>(data.chunk(c));
+    value_arrays.emplace_back(arr->values());
+  }
+  auto flat_column = std::make_shared<Column>(list_type->value_field(), value_arrays);
+  // TODO(ARROW-489): Currently we don't have a Python reference for single columns.
+  //    Storing a reference to the whole Array would be to expensive.
+  PyObject* numpy_array;
+  RETURN_NOT_OK(ConvertColumnToPandas(flat_column, nullptr, &numpy_array));
+
+  PyAcquireGIL lock;
+
+  for (int c = 0; c < data.num_chunks(); c++) {
+    auto arr = std::static_pointer_cast<ListArray>(data.chunk(c));
+
+    const bool has_nulls = data.null_count() > 0;
+    for (int64_t i = 0; i < arr->length(); ++i) {
+      if (has_nulls && arr->IsNull(i)) {
+        Py_INCREF(Py_None);
+        *out_values = Py_None;
+      } else {
+        PyObject* start = PyLong_FromLong(arr->value_offset(i));
+        PyObject* end = PyLong_FromLong(arr->value_offset(i + 1));
+        PyObject* slice = PySlice_New(start, end, NULL);
+        *out_values = PyObject_GetItem(numpy_array, slice);
+        Py_DECREF(start);
+        Py_DECREF(end);
+        Py_DECREF(slice);
+      }
+      ++out_values;
+    }
+  }
+
+  Py_XDECREF(numpy_array);
+  return Status::OK();
+}
+
+template <typename T>
+inline void ConvertNumericNullable(const ChunkedArray& data, T na_value, T* out_values) {
+  for (int c = 0; c < data.num_chunks(); c++) {
+    const std::shared_ptr<Array> arr = data.chunk(c);
+    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
+    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
+
+    const uint8_t* valid_bits = arr->null_bitmap_data();
+
+    if (arr->null_count() > 0) {
+      for (int64_t i = 0; i < arr->length(); ++i) {
+        *out_values++ = BitUtil::BitNotSet(valid_bits, i) ? na_value : in_values[i];
+      }
+    } else {
+      memcpy(out_values, in_values, sizeof(T) * arr->length());
+      out_values += arr->length();
+    }
+  }
+}
+
+template <typename InType, typename OutType>
+inline void ConvertNumericNullableCast(
+    const ChunkedArray& data, OutType na_value, OutType* out_values) {
+  for (int c = 0; c < data.num_chunks(); c++) {
+    const std::shared_ptr<Array> arr = data.chunk(c);
+    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
+    auto in_values = reinterpret_cast<const InType*>(prim_arr->data()->data());
+
+    for (int64_t i = 0; i < arr->length(); ++i) {
+      *out_values++ = arr->IsNull(i) ? na_value : static_cast<OutType>(in_values[i]);
+    }
+  }
+}
+
+template <typename T>
+inline void ConvertDates(const ChunkedArray& data, T na_value, T* out_values) {
+  for (int c = 0; c < data.num_chunks(); c++) {
+    const std::shared_ptr<Array> arr = data.chunk(c);
+    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
+    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
+
+    for (int64_t i = 0; i < arr->length(); ++i) {
+      // There are 1000 * 60 * 60 * 24 = 86400000ms in a day
+      *out_values++ = arr->IsNull(i) ? na_value : in_values[i] / 86400000;
+    }
+  }
+}
+
+template <typename InType, int SHIFT>
+inline void ConvertDatetimeNanos(const ChunkedArray& data, int64_t* out_values) {
+  for (int c = 0; c < data.num_chunks(); c++) {
+    const std::shared_ptr<Array> arr = data.chunk(c);
+    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
+    auto in_values = reinterpret_cast<const InType*>(prim_arr->data()->data());
+
+    for (int64_t i = 0; i < arr->length(); ++i) {
+      *out_values++ = arr->IsNull(i) ? kPandasTimestampNull
+                                     : (static_cast<int64_t>(in_values[i]) * SHIFT);
+    }
+  }
+}
+
+#define CONVERTLISTSLIKE_CASE(ArrowType, ArrowEnum)                \
+  case Type::ArrowEnum:                                            \
+    RETURN_NOT_OK((ConvertListsLike<ArrowType>(col, out_buffer))); \
+    break;
+
+class ObjectBlock : public PandasBlock {
+ public:
+  using PandasBlock::PandasBlock;
+  virtual ~ObjectBlock() {}
+
+  Status Allocate() override { return AllocateNDArray(NPY_OBJECT); }
+
+  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
+      int64_t rel_placement) override {
+    Type::type type = col->type()->type;
+
+    PyObject** out_buffer =
+        reinterpret_cast<PyObject**>(block_data_) + rel_placement * num_rows_;
+
+    const ChunkedArray& data = *col->data().get();
+
+    if (type == Type::BOOL) {
+      RETURN_NOT_OK(ConvertBooleanWithNulls(data, out_buffer));
+    } else if (type == Type::BINARY) {
+      RETURN_NOT_OK(ConvertBinaryLike<BinaryArray>(data, out_buffer));
+    } else if (type == Type::STRING) {
+      RETURN_NOT_OK(ConvertBinaryLike<StringArray>(data, out_buffer));
+    } else if (type == Type::LIST) {
+      auto list_type = std::static_pointer_cast<ListType>(col->type());
+      switch (list_type->value_type()->type) {
+        CONVERTLISTSLIKE_CASE(UInt8Type, UINT8)
+        CONVERTLISTSLIKE_CASE(Int8Type, INT8)
+        CONVERTLISTSLIKE_CASE(UInt16Type, UINT16)
+        CONVERTLISTSLIKE_CASE(Int16Type, INT16)
+        CONVERTLISTSLIKE_CASE(UInt32Type, UINT32)
+        CONVERTLISTSLIKE_CASE(Int32Type, INT32)
+        CONVERTLISTSLIKE_CASE(UInt64Type, UINT64)
+        CONVERTLISTSLIKE_CASE(Int64Type, INT64)
+        CONVERTLISTSLIKE_CASE(TimestampType, TIMESTAMP)
+        CONVERTLISTSLIKE_CASE(FloatType, FLOAT)
+        CONVERTLISTSLIKE_CASE(DoubleType, DOUBLE)
+        CONVERTLISTSLIKE_CASE(StringType, STRING)
+        default: {
+          std::stringstream ss;
+          ss << "Not implemented type for lists: " << list_type->value_type()->ToString();
+          return Status::NotImplemented(ss.str());
+        }
+      }
+    } else {
+      std::stringstream ss;
+      ss << "Unsupported type for object array output: " << col->type()->ToString();
+      return Status::NotImplemented(ss.str());
+    }
+
+    placement_data_[rel_placement] = abs_placement;
+    return Status::OK();
+  }
+};
+
+template <int ARROW_TYPE, typename C_TYPE>
+class IntBlock : public PandasBlock {
+ public:
+  using PandasBlock::PandasBlock;
+
+  Status Allocate() override {
+    return AllocateNDArray(arrow_traits<ARROW_TYPE>::npy_type);
+  }
+
+  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
+      int64_t rel_placement) override {
+    Type::type type = col->type()->type;
+
+    C_TYPE* out_buffer =
+        reinterpret_cast<C_TYPE*>(block_data_) + rel_placement * num_rows_;
+
+    const ChunkedArray& data = *col->data().get();
+
+    if (type != ARROW_TYPE) { return Status::NotImplemented(col->type()->ToString()); }
+
+    ConvertIntegerNoNullsSameType<C_TYPE>(data, out_buffer);
+    placement_data_[rel_placement] = abs_placement;
+    return Status::OK();
+  }
+};
+
+using UInt8Block = IntBlock<Type::UINT8, uint8_t>;
+using Int8Block = IntBlock<Type::INT8, int8_t>;
+using UInt16Block = IntBlock<Type::UINT16, uint16_t>;
+using Int16Block = IntBlock<Type::INT16, int16_t>;
+using UInt32Block = IntBlock<Type::UINT32, uint32_t>;
+using Int32Block = IntBlock<Type::INT32, int32_t>;
+using UInt64Block = IntBlock<Type::UINT64, uint64_t>;
+using Int64Block = IntBlock<Type::INT64, int64_t>;
+
+class Float32Block : public PandasBlock {
+ public:
+  using PandasBlock::PandasBlock;
+
+  Status Allocate() override { return AllocateNDArray(NPY_FLOAT32); }
+
+  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
+      int64_t rel_placement) override {
+    Type::type type = col->type()->type;
+
+    if (type != Type::FLOAT) { return Status::NotImplemented(col->type()->ToString()); }
+
+    float* out_buffer = reinterpret_cast<float*>(block_data_) + rel_placement * num_rows_;
+
+    ConvertNumericNullable<float>(*col->data().get(), NAN, out_buffer);
+    placement_data_[rel_placement] = abs_placement;
+    return Status::OK();
+  }
+};
+
+class Float64Block : public PandasBlock {
+ public:
+  using PandasBlock::PandasBlock;
+
+  Status Allocate() override { return AllocateNDArray(NPY_FLOAT64); }
+
+  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
+      int64_t rel_placement) override {
+    Type::type type = col->type()->type;
+
+    double* out_buffer =
+        reinterpret_cast<double*>(block_data_) + rel_placement * num_rows_;
+
+    const ChunkedArray& data = *col->data().get();
+
+#define INTEGER_CASE(IN_TYPE)                         \
+  ConvertIntegerWithNulls<IN_TYPE>(data, out_buffer); \
+  break;
+
+    switch (type) {
+      case Type::UINT8:
+        INTEGER_CASE(uint8_t);
+      case Type::INT8:
+        INTEGER_CASE(int8_t);
+      case Type::UINT16:
+        INTEGER_CASE(uint16_t);
+      case Type::INT16:
+        INTEGER_CASE(int16_t);
+      case Type::UINT32:
+        INTEGER_CASE(uint32_t);
+      case Type::INT32:
+        INTEGER_CASE(int32_t);
+      case Type::UINT64:
+        INTEGER_CASE(uint64_t);
+      case Type::INT64:
+        INTEGER_CASE(int64_t);
+      case Type::FLOAT:
+        ConvertNumericNullableCast<float, double>(data, NAN, out_buffer);
+        break;
+      case Type::DOUBLE:
+        ConvertNumericNullable<double>(data, NAN, out_buffer);
+        break;
+      default:
+        return Status::NotImplemented(col->type()->ToString());
+    }
+
+#undef INTEGER_CASE
+
+    placement_data_[rel_placement] = abs_placement;
+    return Status::OK();
+  }
+};
+
+class BoolBlock : public PandasBlock {
+ public:
+  using PandasBlock::PandasBlock;
+
+  Status Allocate() override { return AllocateNDArray(NPY_BOOL); }
+
+  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
+      int64_t rel_placement) override {
+    Type::type type = col->type()->type;
+
+    if (type != Type::BOOL) { return Status::NotImplemented(col->type()->ToString()); }
+
+    uint8_t* out_buffer =
+        reinterpret_cast<uint8_t*>(block_data_) + rel_placement * num_rows_;
+
+    ConvertBooleanNoNulls(*col->data().get(), out_buffer);
+    placement_data_[rel_placement] = abs_placement;
+    return Status::OK();
+  }
+};
+
+class DatetimeBlock : public PandasBlock {
+ public:
+  using PandasBlock::PandasBlock;
+
+  Status AllocateDatetime(int ndim) {
+    RETURN_NOT_OK(AllocateNDArray(NPY_DATETIME, ndim));
+
+    PyAcquireGIL lock;
+    auto date_dtype = reinterpret_cast<PyArray_DatetimeDTypeMetaData*>(
+        PyArray_DESCR(reinterpret_cast<PyArrayObject*>(block_arr_.obj()))->c_metadata);
+    date_dtype->meta.base = NPY_FR_ns;
+    return Status::OK();
+  }
+
+  Status Allocate() override { return AllocateDatetime(2); }
+
+  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
+      int64_t rel_placement) override {
+    Type::type type = col->type()->type;
+
+    int64_t* out_buffer =
+        reinterpret_cast<int64_t*>(block_data_) + rel_placement * num_rows_;
+
+    const ChunkedArray& data = *col.get()->data();
+
+    if (type == Type::DATE64) {
+      // Date64Type is millisecond timestamp stored as int64_t
+      // TODO(wesm): Do we want to make sure to zero out the milliseconds?
+      ConvertDatetimeNanos<int64_t, 1000000L>(data, out_buffer);
+    } else if (type == Type::TIMESTAMP) {
+      auto ts_type = static_cast<TimestampType*>(col->type().get());
+
+      if (ts_type->unit == TimeUnit::NANO) {
+        ConvertNumericNullable<int64_t>(data, kPandasTimestampNull, out_buffer);
+      } else if (ts_type->unit == TimeUnit::MICRO) {
+        ConvertDatetimeNanos<int64_t, 1000L>(data, out_buffer);
+      } else if (ts_type->unit == TimeUnit::MILLI) {
+        ConvertDatetimeNanos<int64_t, 1000000L>(data, out_buffer);
+      } else if (ts_type->unit == TimeUnit::SECOND) {
+        ConvertDatetimeNanos<int64_t, 1000000000L>(data, out_buffer);
+      } else {
+        return Status::NotImplemented("Unsupported time unit");
+      }
+    } else {
+      return Status::NotImplemented(col->type()->ToString());
+    }
+
+    placement_data_[rel_placement] = abs_placement;
+    return Status::OK();
+  }
+};
+
+class DatetimeTZBlock : public DatetimeBlock {
+ public:
+  DatetimeTZBlock(const std::string& timezone, int64_t num_rows)
+      : DatetimeBlock(num_rows, 1), timezone_(timezone) {}
+
+  // Like Categorical, the internal ndarray is 1-dimensional
+  Status Allocate() override { return AllocateDatetime(1); }
+
+  Status GetPyResult(PyObject** output) override {
+    PyObject* result = PyDict_New();
+    RETURN_IF_PYERROR();
+
+    PyObject* py_tz = PyUnicode_FromStringAndSize(
+        timezone_.c_str(), static_cast<Py_ssize_t>(timezone_.size()));
+    RETURN_IF_PYERROR();
+
+    PyDict_SetItemString(result, "block", block_arr_.obj());
+    PyDict_SetItemString(result, "timezone", py_tz);
+    PyDict_SetItemString(result, "placement", placement_arr_.obj());
+
+    *output = result;
+
+    return Status::OK();
+  }
+
+ private:
+  std::string timezone_;
+};
+
+template <int ARROW_INDEX_TYPE>
+class CategoricalBlock : public PandasBlock {
+ public:
+  explicit CategoricalBlock(int64_t num_rows) : PandasBlock(num_rows, 1) {}
+
+  Status Allocate() override {
+    constexpr int npy_type = arrow_traits<ARROW_INDEX_TYPE>::npy_type;
+
+    if (!(npy_type == NPY_INT8 || npy_type == NPY_INT16 || npy_type == NPY_INT32 ||
+            npy_type == NPY_INT64)) {
+      return Status::Invalid("Category indices must be signed integers");
+    }
+    return AllocateNDArray(npy_type, 1);
+  }
+
+  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
+      int64_t rel_placement) override {
+    using T = typename arrow_traits<ARROW_INDEX_TYPE>::T;
+
+    T* out_values = reinterpret_cast<T*>(block_data_) + rel_placement * num_rows_;
+
+    const ChunkedArray& data = *col->data().get();
+
+    for (int c = 0; c < data.num_chunks(); c++) {
+      const std::shared_ptr<Array> arr = data.chunk(c);
+      const auto& dict_arr = static_cast<const DictionaryArray&>(*arr);
+      const auto& indices = static_cast<const PrimitiveArray&>(*dict_arr.indices());
+      auto in_values = reinterpret_cast<const T*>(indices.data()->data());
+
+      // Null is -1 in CategoricalBlock
+      for (int i = 0; i < arr->length(); ++i) {
+        *out_values++ = indices.IsNull(i) ? -1 : in_values[i];
+      }
+    }
+
+    placement_data_[rel_placement] = abs_placement;
+
+    auto dict_type = static_cast<const DictionaryType*>(col->type().get());
+
+    PyObject* dict;
+    RETURN_NOT_OK(ConvertArrayToPandas(dict_type->dictionary(), nullptr, &dict));
+    dictionary_.reset(dict);
+
+    return Status::OK();
+  }
+
+  Status GetPyResult(PyObject** output) override {
+    PyObject* result = PyDict_New();
+    RETURN_IF_PYERROR();
+
+    PyDict_SetItemString(result, "block", block_arr_.obj());
+    PyDict_SetItemString(result, "dictionary", dictionary_.obj());
+    PyDict_SetItemString(result, "placement", placement_arr_.obj());
+
+    *output = result;
+
+    return Status::OK();
+  }
+
+ protected:
+  OwnedRef dictionary_;
+};
+
+Status MakeBlock(PandasBlock::type type, int64_t num_rows, int num_columns,
+    std::shared_ptr<PandasBlock>* block) {
+#define BLOCK_CASE(NAME, TYPE)                              \
+  case PandasBlock::NAME:                                   \
+    *block = std::make_shared<TYPE>(num_rows, num_columns); \
+    break;
+
+  switch (type) {
+    BLOCK_CASE(OBJECT, ObjectBlock);
+    BLOCK_CASE(UINT8, UInt8Block);
+    BLOCK_CASE(INT8, Int8Block);
+    BLOCK_CASE(UINT16, UInt16Block);
+    BLOCK_CASE(INT16, Int16Block);
+    BLOCK_CASE(UINT32, UInt32Block);
+    BLOCK_CASE(INT32, Int32Block);
+    BLOCK_CASE(UINT64, UInt64Block);
+    BLOCK_CASE(INT64, Int64Block);
+    BLOCK_CASE(FLOAT, Float32Block);
+    BLOCK_CASE(DOUBLE, Float64Block);
+    BLOCK_CASE(BOOL, BoolBlock);
+    BLOCK_CASE(DATETIME, DatetimeBlock);
+    default:
+      return Status::NotImplemented("Unsupported block type");
+  }
+
+#undef BLOCK_CASE
+
+  return (*block)->Allocate();
+}
+
+static inline Status MakeCategoricalBlock(const std::shared_ptr<DataType>& type,
+    int64_t num_rows, std::shared_ptr<PandasBlock>* block) {
+  // All categoricals become a block with a single column
+  auto dict_type = static_cast<const DictionaryType*>(type.get());
+  switch (dict_type->index_type()->type) {
+    case Type::INT8:
+      *block = std::make_shared<CategoricalBlock<Type::INT8>>(num_rows);
+      break;
+    case Type::INT16:
+      *block = std::make_shared<CategoricalBlock<Type::INT16>>(num_rows);
+      break;
+    case Type::INT32:
+      *block = std::make_shared<CategoricalBlock<Type::INT32>>(num_rows);
+      break;
+    case Type::INT64:
+      *block = std::make_shared<CategoricalBlock<Type::INT64>>(num_rows);
+      break;
+    default: {
+      std::stringstream ss;
+      ss << "Categorical index type not implemented: "
+         << dict_type->index_type()->ToString();
+      return Status::NotImplemented(ss.str());
+    }
+  }
+  return (*block)->Allocate();
+}
+
+using BlockMap = std::unordered_map<int, std::shared_ptr<PandasBlock>>;
+
+// Construct the exact pandas 0.x "BlockManager" memory layout
+//
+// * For each column determine the correct output pandas type
+// * Allocate 2D blocks (ncols x nrows) for each distinct data type in output
+// * Allocate  block placement arrays
+// * Write Arrow columns out into each slice of memory; populate block
+// * placement arrays as we go
+class DataFrameBlockCreator {
+ public:
+  explicit DataFrameBlockCreator(const std::shared_ptr<Table>& table) : table_(table) {}
+
+  Status Convert(int nthreads, PyObject** output) {
+    column_types_.resize(table_->num_columns());
+    column_block_placement_.resize(table_->num_columns());
+    type_counts_.clear();
+    blocks_.clear();
+
+    RETURN_NOT_OK(CreateBlocks());
+    RETURN_NOT_OK(WriteTableToBlocks(nthreads));
+
+    return GetResultList(output);
+  }
+
+  Status CreateBlocks() {
+    for (int i = 0; i < table_->num_columns(); ++i) {
+      std::shared_ptr<Column> col = table_->column(i);
+      PandasBlock::type output_type;
+
+      Type::type column_type = col->type()->type;
+      switch (column_type) {
+        case Type::BOOL:
+          output_type = col->null_count() > 0 ? PandasBlock::OBJECT : PandasBlock::BOOL;
+          break;
+        case Type::UINT8:
+          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::UINT8;
+          break;
+        case Type::INT8:
+          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::INT8;
+          break;
+        case Type::UINT16:
+          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::UINT16;
+          break;
+        case Type::INT16:
+          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::INT16;
+          break;
+        case Type::UINT32:
+          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::UINT32;
+          break;
+        case Type::INT32:
+          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::INT32;
+          break;
+        case Type::INT64:
+          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::INT64;
+          break;
+        case Type::UINT64:
+          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::UINT64;
+          break;
+        case Type::FLOAT:
+          output_type = PandasBlock::FLOAT;
+          break;
+        case Type::DOUBLE:
+          output_type = PandasBlock::DOUBLE;
+          break;
+        case Type::STRING:
+        case Type::BINARY:
+          output_type = PandasBlock::OBJECT;
+          break;
+        case Type::DATE64:
+          output_type = PandasBlock::DATETIME;
+          break;
+        case Type::TIMESTAMP: {
+          const auto& ts_type = static_cast<const TimestampType&>(*col->type());
+          if (ts_type.timezone != "") {
+            output_type = PandasBlock::DATETIME_WITH_TZ;
+          } else {
+            output_type = PandasBlock::DATETIME;
+          }
+        } break;
+        case Type::LIST: {
+          auto list_type = std::static_pointer_cast<ListType>(col->type());
+          if (!ListTypeSupported(list_type->value_type()->type)) {
+            std::stringstream ss;
+            ss << "Not implemented type for lists: "
+               << list_type->value_type()->ToString();
+            return Status::NotImplemented(ss.str());
+          }
+          output_type = PandasBlock::OBJECT;
+        } break;
+        case Type::DICTIONARY:
+          output_type = PandasBlock::CATEGORICAL;
+          break;
+        default:
+          return Status::NotImplemented(col->type()->ToString());
+      }
+
+      int block_placement = 0;
+      std::shared_ptr<PandasBlock> block;
+      if (output_type == PandasBlock::CATEGORICAL) {
+        RETURN_NOT_OK(MakeCategoricalBlock(col->type(), table_->num_rows(), &block));
+        categorical_blocks_[i] = block;
+      } else if (output_type == PandasBlock::DATETIME_WITH_TZ) {
+        const auto& ts_type = static_cast<const TimestampType&>(*col->type());
+        block = std::make_shared<DatetimeTZBlock>(ts_type.timezone, table_->num_rows());
+        RETURN_NOT_OK(block->Allocate());
+        datetimetz_blocks_[i] = block;
+      } else {
+        auto it = type_counts_.find(output_type);
+        if (it != type_counts_.end()) {
+          block_placement = it->second;
+          // Increment count
+          it->second += 1;
+        } else {
+          // Add key to map
+          type_counts_[output_type] = 1;
+        }
+      }
+
+      column_types_[i] = output_type;
+      column_block_placement_[i] = block_placement;
+    }
+
+    // Create normal non-categorical blocks
+    for (const auto& it : type_counts_) {
+      PandasBlock::type type = static_cast<PandasBlock::type>(it.first);
+      std::shared_ptr<PandasBlock> block;
+      RETURN_NOT_OK(MakeBlock(type, table_->num_rows(), it.second, &block));
+      blocks_[type] = block;
+    }
+    return Status::OK();
+  }
+
+  Status WriteTableToBlocks(int nthreads) {
+    auto WriteColumn = [this](int i) {
+      std::shared_ptr<Column> col = this->table_->column(i);
+      PandasBlock::type output_type = this->column_types_[i];
+
+      int rel_placement = this->column_block_placement_[i];
+
+      std::shared_ptr<PandasBlock> block;
+      if (output_type == PandasBlock::CATEGORICAL) {
+        auto it = this->categorical_blocks_.find(i);
+        if (it == this->blocks_.end()) {
+          return Status::KeyError("No categorical block allocated");
+        }
+        block = it->second;
+      } else if (output_type == PandasBlock::DATETIME_WITH_TZ) {
+        auto it = this->datetimetz_blocks_.find(i);
+        if (it == this->datetimetz_blocks_.end()) {
+          return Status::KeyError("No datetimetz block allocated");
+        }
+        block = it->second;
+      } else {
+        auto it = this->blocks_.find(output_type);
+        if (it == this->blocks_.end()) { return Status::KeyError("No block allocated"); }
+        block = it->second;
+      }
+      return block->Write(col, i, rel_placement);
+    };
+
+    nthreads = std::min<int>(nthreads, table_->num_columns());
+
+    if (nthreads == 1) {
+      for (int i = 0; i < table_->num_columns(); ++i) {
+        RETURN_NOT_OK(WriteColumn(i));
+      }
+    } else {
+      std::vector<std::thread> thread_pool;
+      thread_pool.reserve(nthreads);
+      std::atomic<int> task_counter(0);
+
+      std::mutex error_mtx;
+      bool error_occurred = false;
+      Status error;
+
+      for (int thread_id = 0; thread_id < nthreads; ++thread_id) {
+        thread_pool.emplace_back(
+            [this, &error, &error_occurred, &error_mtx, &task_counter, &WriteColumn]() {
+              int column_num;
+              while (!error_occurred) {
+                column_num = task_counter.fetch_add(1);
+                if (column_num >= this->table_->num_columns()) { break; }
+                Status s = WriteColumn(column_num);
+                if (!s.ok()) {
+                  std::lock_guard<std::mutex> lock(error_mtx);
+                  error_occurred = true;
+                  error = s;
+                  break;
+                }
+              }
+            });
+      }
+      for (auto&& thread : thread_pool) {
+        thread.join();
+      }
+
+      if (error_occurred) { return error; }
+    }
+    return Status::OK();
+  }
+
+  Status AppendBlocks(const BlockMap& blocks, PyObject* list) {
+    for (const auto& it : blocks) {
+      PyObject* item;
+      RETURN_NOT_OK(it.second->GetPyResult(&item));
+      if (PyList_Append(list, item) < 0) { RETURN_IF_PYERROR(); }
+    }
+    return Status::OK();
+  }
+
+  Status GetResultList(PyObject** out) {
+    PyAcquireGIL lock;
+
+    PyObject* result = PyList_New(0);
+    RETURN_IF_PYERROR();
+
+    RETURN_NOT_OK(AppendBlocks(blocks_, result));
+    RETURN_NOT_OK(AppendBlocks(categorical_blocks_, result));
+    RETURN_NOT_OK(AppendBlocks(datetimetz_blocks_, result));
+
+    *out = result;
+    return Status::OK();
+  }
+
+ private:
+  std::shared_ptr<Table> table_;
+
+  // column num -> block type id
+  std::vector<PandasBlock::type> column_types_;
+
+  // column num -> relative placement within internal block
+  std::vector<int> column_block_placement_;
+
+  // block type -> type count
+  std::unordered_map<int, int> type_counts_;
+
+  // block type -> block
+  BlockMap blocks_;
+
+  // column number -> categorical block
+  BlockMap categorical_blocks_;
+
+  // column number -> datetimetz block
+  BlockMap datetimetz_blocks_;
+};
+
+class ArrowDeserializer {
+ public:
+  ArrowDeserializer(const std::shared_ptr<Column>& col, PyObject* py_ref)
+      : col_(col), data_(*col->data().get()), py_ref_(py_ref) {}
+
+  Status AllocateOutput(int type) {
+    PyAcquireGIL lock;
+
+    npy_intp dims[1] = {col_->length()};
+    result_ = PyArray_SimpleNew(1, dims, type);
+    arr_ = reinterpret_cast<PyArrayObject*>(result_);
+
+    if (arr_ == NULL) {
+      // Error occurred, trust that SimpleNew set the error state
+      return Status::OK();
+    }
+
+    set_numpy_metadata(type, col_->type().get(), arr_);
+
+    return Status::OK();
+  }
+
+  template <int TYPE>
+  Status ConvertValuesZeroCopy(int npy_type, std::shared_ptr<Array> arr) {
+    typedef typename arrow_traits<TYPE>::T T;
+
+    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
+    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
+
+    // Zero-Copy. We can pass the data pointer directly to NumPy.
+    void* data = const_cast<T*>(in_values);
+
+    PyAcquireGIL lock;
+
+    // Zero-Copy. We can pass the data pointer directly to NumPy.
+    npy_intp dims[1] = {col_->length()};
+    result_ = PyArray_SimpleNewFromData(1, dims, npy_type, data);
+    arr_ = reinterpret_cast<PyArrayObject*>(result_);
+
+    if (arr_ == NULL) {
+      // Error occurred, trust that SimpleNew set the error state
+      return Status::OK();
+    }
+
+    set_numpy_metadata(npy_type, col_->type().get(), arr_);
+
+    if (PyArray_SetBaseObject(arr_, py_ref_) == -1) {
+      // Error occurred, trust that SetBaseObject set the error state
+      return Status::OK();
+    } else {
+      // PyArray_SetBaseObject steals our reference to py_ref_
+      Py_INCREF(py_ref_);
+    }
+
+    // Arrow data is immutable.
+    PyArray_CLEARFLAGS(arr_, NPY_ARRAY_WRITEABLE);
+
+    return Status::OK();
+  }
+
+  // ----------------------------------------------------------------------
+  // Allocate new array and deserialize. Can do a zero copy conversion for some
+  // types
+
+  Status Convert(PyObject** out) {
+#define CONVERT_CASE(TYPE)                      \
+  case Type::TYPE: {                            \
+    RETURN_NOT_OK(ConvertValues<Type::TYPE>()); \
+  } break;
+
+    switch (col_->type()->type) {
+      CONVERT_CASE(BOOL);
+      CONVERT_CASE(INT8);
+      CONVERT_CASE(INT16);
+      CONVERT_CASE(INT32);
+      CONVERT_CASE(INT64);
+      CONVERT_CASE(UINT8);
+      CONVERT_CASE(UINT16);
+      CONVERT_CASE(UINT32);
+      CONVERT_CASE(UINT64);
+      CONVERT_CASE(FLOAT);
+      CONVERT_CASE(DOUBLE);
+      CONVERT_CASE(BINARY);
+      CONVERT_CASE(STRING);
+      CONVERT_CASE(DATE64);
+      CONVERT_CASE(TIMESTAMP);
+      CONVERT_CASE(DICTIONARY);
+      CONVERT_CASE(LIST);
+      default: {
+        std::stringstream ss;
+        ss << "Arrow type reading not implemented for " << col_->type()->ToString();
+        return Status::NotImplemented(ss.str());
+      }
+    }
+
+#undef CONVERT_CASE
+
+    *out = result_;
+    return Status::OK();
+  }
+
+  template <int TYPE>
+  inline typename std::enable_if<
+      (TYPE != Type::DATE64) & arrow_traits<TYPE>::is_numeric_nullable, Status>::type
+  ConvertValues() {
+    typedef typename arrow_traits<TYPE>::T T;
+    int npy_type = arrow_traits<TYPE>::npy_type;
+
+    if (data_.num_chunks() == 1 && data_.null_count() == 0 && py_ref_ != nullptr) {
+      return ConvertValuesZeroCopy<TYPE>(npy_type, data_.chunk(0));
+    }
+
+    RETURN_NOT_OK(AllocateOutput(npy_type));
+    auto out_values = reinterpret_cast<T*>(PyArray_DATA(arr_));
+    ConvertNumericNullable<T>(data_, arrow_traits<TYPE>::na_value, out_values);
+
+    return Status::OK();
+  }
+
+  template <int TYPE>
+  inline typename std::enable_if<TYPE == Type::DATE64, Status>::type ConvertValues() {
+    typedef typename arrow_traits<TYPE>::T T;
+
+    RETURN_NOT_OK(AllocateOutput(arrow_traits<TYPE>::npy_type));
+    auto out_values = reinterpret_cast<T*>(PyArray_DATA(arr_));
+    ConvertDates<T>(data_, arrow_traits<TYPE>::na_value, out_values);
+    return Status::OK();
+  }
+
+  // Integer specialization
+  template <int TYPE>
+  inline
+      typename std::enable_if<arrow_traits<TYPE>::is_numeric_not_nullable, Status>::type
+      ConvertValues() {
+    typedef typename arrow_traits<TYPE>::T T;
+    int npy_type = arrow_traits<TYPE>::npy_type;
+
+    if (data_.num_chunks() == 1 && data_.null_count() == 0 && py_ref_ != nullptr) {
+      return ConvertValuesZeroCopy<TYPE>(npy_type, data_.chunk(0));
+    }
+
+    if (data_.null_count() > 0) {
+      RETURN_NOT_OK(AllocateOutput(NPY_FLOAT64));
+      auto out_values = reinterpret_cast<double*>(PyArray_DATA(arr_));
+      ConvertIntegerWithNulls<T>(data_, out_values);
+    } else {
+      RETURN_NOT_OK(AllocateOutput(arrow_traits<TYPE>::npy_type));
+      auto out_values = reinterpret_cast<T*>(PyArray_DATA(arr_));
+      ConvertIntegerNoNullsSameType<T>(data_, out_values);
+    }
+
+    return Status::OK();
+  }
+
+  // Boolean specialization
+  template <int TYPE>
+  inline typename std::enable_if<arrow_traits<TYPE>::is_boolean, Status>::type
+  ConvertValues() {
+    if (data_.null_count() > 0) {
+      RETURN_NOT_OK(AllocateOutput(NPY_OBJECT));
+      auto out_values = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
+      RETURN_NOT_OK(ConvertBooleanWithNulls(data_, out_values));
+    } else {
+      RETURN_NOT_OK(AllocateOutput(arrow_traits<TYPE>::npy_type));
+      auto out_values = reinterpret_cast<uint8_t*>(PyArray_DATA(arr_));
+      ConvertBooleanNoNulls(data_, out_values);
+    }
+    return Status::OK();
+  }
+
+  // UTF8 strings
+  template <int TYPE>
+  inline typename std::enable_if<TYPE == Type::STRING, Status>::type ConvertValues() {
+    RETURN_NOT_OK(AllocateOutput(NPY_OBJECT));
+    auto out_values = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
+    return ConvertBinaryLike<StringArray>(data_, out_values);
+  }
+
+  template <int T2>
+  inline typename std::enable_if<T2 == Type::BINARY, Status>::type ConvertValues() {
+    RETURN_NOT_OK(AllocateOutput(NPY_OBJECT));
+    auto out_values = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
+    return ConvertBinaryLike<BinaryArray>(data_, out_values);
+  }
+
+#define CONVERTVALUES_LISTSLIKE_CASE(ArrowType, ArrowEnum) \
+  case Type::ArrowEnum:                                    \
+    return ConvertListsLike<ArrowType>(col_, out_values);
+
+  template <int T2>
+  inline typename std::enable_if<T2 == Type::LIST, Status>::type ConvertValues() {
+    RETURN_NOT_OK(AllocateOutput(NPY_OBJECT));
+    auto out_values = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
+    auto list_type = std::static_pointer_cast<ListType>(col_->type());
+    switch (list_type->value_type()->type) {
+      CONVERTVALUES_LISTSLIKE_CASE(UInt8Type, UINT8)
+      CONVERTVALUES_LISTSLIKE_CASE(Int8Type, INT8)
+      CONVERTVALUES_LISTSLIKE_CASE(UInt16Type, UINT16)
+      CONVERTVALUES_LISTSLIKE_CASE(Int16Type, INT16)
+      CONVERTVALUES_LISTSLIKE_CASE(UInt32Type, UINT32)
+      CONVERTVALUES_LISTSLIKE_CASE(Int32Type, INT32)
+      CONVERTVALUES_LISTSLIKE_CASE(UInt64Type, UINT64)
+      CONVERTVALUES_LISTSLIKE_CASE(Int64Type, INT64)
+      CONVERTVALUES_LISTSLIKE_CASE(TimestampType, TIMESTAMP)
+      CONVERTVALUES_LISTSLIKE_CASE(FloatType, FLOAT)
+      CONVERTVALUES_LISTSLIKE_CASE(DoubleType, DOUBLE)
+      CONVERTVALUES_LISTSLIKE_CASE(StringType, STRING)
+      default: {
+        std::stringstream ss;
+        ss << "Not implemented type for lists: " << list_type->value_type()->ToString();
+        return Status::NotImplemented(ss.str());
+      }
+    }
+  }
+
+  template <int TYPE>
+  inline typename std::enable_if<TYPE == Type::DICTIONARY, Status>::type ConvertValues() {
+    std::shared_ptr<PandasBlock> block;
+    RETURN_NOT_OK(MakeCategoricalBlock(col_->type(), col_->length(), &block));
+    RETURN_NOT_OK(block->Write(col_, 0, 0));
+
+    auto dict_type = static_cast<const DictionaryType*>(col_->type().get());
+
+    PyAcquireGIL lock;
+    result_ = PyDict_New();
+    RETURN_IF_PYERROR();
+
+    PyObject* dictionary;
+    RETURN_NOT_OK(ConvertArrayToPandas(dict_type->dictionary(), nullptr, &dictionary));
+
+    PyDict_SetItemString(result_, "indices", block->block_arr());
+    PyDict_SetItemString(result_, "dictionary", dictionary);
+
+    return Status::OK();
+  }
+
+ private:
+  std::shared_ptr<Column> col_;
+  const ChunkedArray& data_;
+  PyObject* py_ref_;
+  PyArrayObject* arr_;
+  PyObject* result_;
+};
+
+Status ConvertArrayToPandas(
+    const std::shared_ptr<Array>& arr, PyObject* py_ref, PyObject** out) {
+  static std::string dummy_name = "dummy";
+  auto field = std::make_shared<Field>(dummy_name, arr->type());
+  auto col = std::make_shared<Column>(field, arr);
+  return ConvertColumnToPandas(col, py_ref, out);
+}
+
+Status ConvertColumnToPandas(
+    const std::shared_ptr<Column>& col, PyObject* py_ref, PyObject** out) {
+  ArrowDeserializer converter(col, py_ref);
+  return converter.Convert(out);
+}
+
+Status ConvertTableToPandas(
+    const std::shared_ptr<Table>& table, int nthreads, PyObject** out) {
+  DataFrameBlockCreator helper(table);
+  return helper.Convert(nthreads, out);
+}
+
+}  // namespace py
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/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
new file mode 100644
index 0000000..a33741e
--- /dev/null
+++ b/cpp/src/arrow/python/pandas_convert.h
@@ -0,0 +1,79 @@
+// 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.
+
+// Functions for converting between pandas's NumPy-based data representation
+// and Arrow data structures
+
+#ifndef ARROW_PYTHON_ADAPTERS_PANDAS_H
+#define ARROW_PYTHON_ADAPTERS_PANDAS_H
+
+#include <Python.h>
+
+#include <memory>
+
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+
+class Array;
+class Column;
+class DataType;
+class MemoryPool;
+class Status;
+class Table;
+
+namespace py {
+
+ARROW_EXPORT
+Status ConvertArrayToPandas(
+    const std::shared_ptr<Array>& arr, PyObject* py_ref, PyObject** out);
+
+ARROW_EXPORT
+Status ConvertColumnToPandas(
+    const std::shared_ptr<Column>& col, PyObject* py_ref, PyObject** out);
+
+struct PandasOptions {
+  bool strings_to_categorical;
+};
+
+// Convert a whole table as efficiently as possible to a pandas.DataFrame.
+//
+// The returned Python object is a list of tuples consisting of the exact 2D
+// BlockManager structure of the pandas.DataFrame used as of pandas 0.19.x.
+//
+// tuple item: (indices: ndarray[int32], block: ndarray[TYPE, ndim=2])
+ARROW_EXPORT
+Status ConvertTableToPandas(
+    const std::shared_ptr<Table>& table, int nthreads, PyObject** out);
+
+ARROW_EXPORT
+Status PandasDtypeToArrow(PyObject* dtype, std::shared_ptr<DataType>* out);
+
+ARROW_EXPORT
+Status PandasToArrow(MemoryPool* pool, PyObject* ao, PyObject* mo,
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out);
+
+/// Convert dtype=object arrays. If target data type is not known, pass a type
+/// with nullptr
+ARROW_EXPORT
+Status PandasObjectsToArrow(MemoryPool* pool, PyObject* ao, PyObject* mo,
+    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out);
+
+}  // namespace py
+}  // namespace arrow
+
+#endif  // ARROW_PYTHON_ADAPTERS_PANDAS_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/type_traits.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/type_traits.h b/cpp/src/arrow/python/type_traits.h
new file mode 100644
index 0000000..f78dc36
--- /dev/null
+++ b/cpp/src/arrow/python/type_traits.h
@@ -0,0 +1,213 @@
+// 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 <Python.h>
+
+#include <cstdint>
+#include <limits>
+
+#include "arrow/python/numpy_interop.h"
+
+#include "arrow/builder.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace py {
+
+template <int TYPE>
+struct npy_traits {};
+
+template <>
+struct npy_traits<NPY_BOOL> {
+  typedef uint8_t value_type;
+  using TypeClass = BooleanType;
+  using BuilderClass = BooleanBuilder;
+
+  static constexpr bool supports_nulls = false;
+  static inline bool isnull(uint8_t v) { return false; }
+};
+
+#define NPY_INT_DECL(TYPE, CapType, T)               \
+  template <>                                        \
+  struct npy_traits<NPY_##TYPE> {                    \
+    typedef T value_type;                            \
+    using TypeClass = CapType##Type;                 \
+    using BuilderClass = CapType##Builder;           \
+                                                     \
+    static constexpr bool supports_nulls = false;    \
+    static inline bool isnull(T v) { return false; } \
+  };
+
+NPY_INT_DECL(INT8, Int8, int8_t);
+NPY_INT_DECL(INT16, Int16, int16_t);
+NPY_INT_DECL(INT32, Int32, int32_t);
+NPY_INT_DECL(INT64, Int64, int64_t);
+
+NPY_INT_DECL(UINT8, UInt8, uint8_t);
+NPY_INT_DECL(UINT16, UInt16, uint16_t);
+NPY_INT_DECL(UINT32, UInt32, uint32_t);
+NPY_INT_DECL(UINT64, UInt64, uint64_t);
+
+#if NPY_INT64 != NPY_LONGLONG
+NPY_INT_DECL(LONGLONG, Int64, int64_t);
+NPY_INT_DECL(ULONGLONG, UInt64, uint64_t);
+#endif
+
+template <>
+struct npy_traits<NPY_FLOAT32> {
+  typedef float value_type;
+  using TypeClass = FloatType;
+  using BuilderClass = FloatBuilder;
+
+  static constexpr bool supports_nulls = true;
+
+  static inline bool isnull(float v) { return v != v; }
+};
+
+template <>
+struct npy_traits<NPY_FLOAT64> {
+  typedef double value_type;
+  using TypeClass = DoubleType;
+  using BuilderClass = DoubleBuilder;
+
+  static constexpr bool supports_nulls = true;
+
+  static inline bool isnull(double v) { return v != v; }
+};
+
+template <>
+struct npy_traits<NPY_DATETIME> {
+  typedef int64_t value_type;
+  using TypeClass = TimestampType;
+  using BuilderClass = TimestampBuilder;
+
+  static constexpr bool supports_nulls = true;
+
+  static inline bool isnull(int64_t v) {
+    // NaT = -2**63
+    // = -0x8000000000000000
+    // = -9223372036854775808;
+    // = std::numeric_limits<int64_t>::min()
+    return v == std::numeric_limits<int64_t>::min();
+  }
+};
+
+template <>
+struct npy_traits<NPY_OBJECT> {
+  typedef PyObject* value_type;
+  static constexpr bool supports_nulls = true;
+};
+
+template <int TYPE>
+struct arrow_traits {};
+
+template <>
+struct arrow_traits<Type::BOOL> {
+  static constexpr int npy_type = NPY_BOOL;
+  static constexpr bool supports_nulls = false;
+  static constexpr bool is_boolean = true;
+  static constexpr bool is_numeric_not_nullable = false;
+  static constexpr bool is_numeric_nullable = false;
+};
+
+#define INT_DECL(TYPE)                                     \
+  template <>                                              \
+  struct arrow_traits<Type::TYPE> {                        \
+    static constexpr int npy_type = NPY_##TYPE;            \
+    static constexpr bool supports_nulls = false;          \
+    static constexpr double na_value = NAN;                \
+    static constexpr bool is_boolean = false;              \
+    static constexpr bool is_numeric_not_nullable = true;  \
+    static constexpr bool is_numeric_nullable = false;     \
+    typedef typename npy_traits<NPY_##TYPE>::value_type T; \
+  };
+
+INT_DECL(INT8);
+INT_DECL(INT16);
+INT_DECL(INT32);
+INT_DECL(INT64);
+INT_DECL(UINT8);
+INT_DECL(UINT16);
+INT_DECL(UINT32);
+INT_DECL(UINT64);
+
+template <>
+struct arrow_traits<Type::FLOAT> {
+  static constexpr int npy_type = NPY_FLOAT32;
+  static constexpr bool supports_nulls = true;
+  static constexpr float na_value = NAN;
+  static constexpr bool is_boolean = false;
+  static constexpr bool is_numeric_not_nullable = false;
+  static constexpr bool is_numeric_nullable = true;
+  typedef typename npy_traits<NPY_FLOAT32>::value_type T;
+};
+
+template <>
+struct arrow_traits<Type::DOUBLE> {
+  static constexpr int npy_type = NPY_FLOAT64;
+  static constexpr bool supports_nulls = true;
+  static constexpr double na_value = NAN;
+  static constexpr bool is_boolean = false;
+  static constexpr bool is_numeric_not_nullable = false;
+  static constexpr bool is_numeric_nullable = true;
+  typedef typename npy_traits<NPY_FLOAT64>::value_type T;
+};
+
+static constexpr int64_t kPandasTimestampNull = std::numeric_limits<int64_t>::min();
+
+template <>
+struct arrow_traits<Type::TIMESTAMP> {
+  static constexpr int npy_type = NPY_DATETIME;
+  static constexpr bool supports_nulls = true;
+  static constexpr int64_t na_value = kPandasTimestampNull;
+  static constexpr bool is_boolean = false;
+  static constexpr bool is_numeric_not_nullable = false;
+  static constexpr bool is_numeric_nullable = true;
+  typedef typename npy_traits<NPY_DATETIME>::value_type T;
+};
+
+template <>
+struct arrow_traits<Type::DATE64> {
+  static constexpr int npy_type = NPY_DATETIME;
+  static constexpr bool supports_nulls = true;
+  static constexpr int64_t na_value = kPandasTimestampNull;
+  static constexpr bool is_boolean = false;
+  static constexpr bool is_numeric_not_nullable = false;
+  static constexpr bool is_numeric_nullable = true;
+  typedef typename npy_traits<NPY_DATETIME>::value_type T;
+};
+
+template <>
+struct arrow_traits<Type::STRING> {
+  static constexpr int npy_type = NPY_OBJECT;
+  static constexpr bool supports_nulls = true;
+  static constexpr bool is_boolean = false;
+  static constexpr bool is_numeric_not_nullable = false;
+  static constexpr bool is_numeric_nullable = false;
+};
+
+template <>
+struct arrow_traits<Type::BINARY> {
+  static constexpr int npy_type = NPY_OBJECT;
+  static constexpr bool supports_nulls = true;
+  static constexpr bool is_boolean = false;
+  static constexpr bool is_numeric_not_nullable = false;
+  static constexpr bool is_numeric_nullable = false;
+};
+
+}  // namespace py
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/util/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/util/CMakeLists.txt b/cpp/src/arrow/python/util/CMakeLists.txt
new file mode 100644
index 0000000..4cc20f6
--- /dev/null
+++ b/cpp/src/arrow/python/util/CMakeLists.txt
@@ -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.
+
+#######################################
+# arrow/python_test_main
+#######################################
+
+if (PYARROW_BUILD_TESTS)
+  add_library(arrow/python_test_main STATIC
+	test_main.cc)
+
+  if (APPLE)
+	target_link_libraries(arrow/python_test_main
+      gtest
+      dl)
+	set_target_properties(arrow/python_test_main
+      PROPERTIES LINK_FLAGS "-undefined dynamic_lookup")
+  else()
+	target_link_libraries(arrow/python_test_main
+      gtest
+      pthread
+      dl
+	  )
+  endif()
+endif()

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/util/datetime.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/util/datetime.h b/cpp/src/arrow/python/util/datetime.h
new file mode 100644
index 0000000..f704a96
--- /dev/null
+++ b/cpp/src/arrow/python/util/datetime.h
@@ -0,0 +1,42 @@
+// 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.
+
+#ifndef PYARROW_UTIL_DATETIME_H
+#define PYARROW_UTIL_DATETIME_H
+
+#include <Python.h>
+#include <datetime.h>
+
+namespace arrow {
+namespace py {
+
+inline int64_t PyDate_to_ms(PyDateTime_Date* pydate) {
+  struct tm date = {0};
+  date.tm_year = PyDateTime_GET_YEAR(pydate) - 1900;
+  date.tm_mon = PyDateTime_GET_MONTH(pydate) - 1;
+  date.tm_mday = PyDateTime_GET_DAY(pydate);
+  struct tm epoch = {0};
+  epoch.tm_year = 70;
+  epoch.tm_mday = 1;
+  // Milliseconds since the epoch
+  return lrint(difftime(mktime(&date), mktime(&epoch)) * 1000);
+}
+
+}  // namespace py
+}  // namespace arrow
+
+#endif  // PYARROW_UTIL_DATETIME_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/util/test_main.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/util/test_main.cc b/cpp/src/arrow/python/util/test_main.cc
new file mode 100644
index 0000000..c83514d
--- /dev/null
+++ b/cpp/src/arrow/python/util/test_main.cc
@@ -0,0 +1,36 @@
+// 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 <Python.h>
+
+#include <gtest/gtest.h>
+
+#include "arrow/python/do_import_numpy.h"
+#include "arrow/python/numpy_interop.h"
+
+int main(int argc, char** argv) {
+  ::testing::InitGoogleTest(&argc, argv);
+
+  Py_Initialize();
+  arrow::py::import_numpy();
+
+  int ret = RUN_ALL_TESTS();
+
+  Py_Finalize();
+
+  return ret;
+}


[5/5] arrow git commit: ARROW-341: [Python] Move pyarrow's C++ code to the main C++ source tree, install libarrow_python and headers

Posted by we...@apache.org.
ARROW-341: [Python] Move pyarrow's C++ code to the main C++ source tree, install libarrow_python and headers

This will enable third parties to link to `libarrow_python`.

For now, the pyarrow build system continues to use CMake -- for the purpose of resolving the thirdparty toolchain we may or may not want to go completely to distutils, but we can sort that out later.

Author: Wes McKinney <we...@twosigma.com>

Closes #440 from wesm/ARROW-341 and squashes the following commits:

193bc51 [Wes McKinney] Ensure that '-undefined dynamic_lookup' is passed when linking shared library on OS X
a93496b [Wes McKinney] Add missing backslash
7620f50 [Wes McKinney] Fix cpplint issues
0617c69 [Wes McKinney] Fix LD_LIBRARY_PATH, ARROW_HOME
090c78c [Wes McKinney] Build Arrow library stack specific to active Python version
10e4626 [Wes McKinney] Get Python test suite passing again
cfb7f44 [Wes McKinney] Remove print statement
c1e63dc [Wes McKinney] Scrubbing python/CMakeLists.txt
b80b153 [Wes McKinney] Cleanup, build pandas-test within main test suite
7ef1f81 [Wes McKinney] Start moving python/src/pyarrow tp cpp/src/arrow/python


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

Branch: refs/heads/master
Commit: 3aac4adef11345f211e4c66467ff758cbc397e43
Parents: 6d4e862
Author: Wes McKinney <we...@twosigma.com>
Authored: Sun Mar 26 11:45:38 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Sun Mar 26 11:45:38 2017 -0400

----------------------------------------------------------------------
 ci/travis_script_python.sh                   |   26 +-
 cpp/CMakeLists.txt                           |  115 +-
 cpp/cmake_modules/BuildUtils.cmake           |   88 +-
 cpp/cmake_modules/FindNumPy.cmake            |  100 ++
 cpp/cmake_modules/FindPythonLibsNew.cmake    |  241 +++
 cpp/src/arrow/python/CMakeLists.txt          |   93 +
 cpp/src/arrow/python/api.h                   |   27 +
 cpp/src/arrow/python/builtin_convert.cc      |  527 ++++++
 cpp/src/arrow/python/builtin_convert.h       |   54 +
 cpp/src/arrow/python/common.cc               |   68 +
 cpp/src/arrow/python/common.h                |  139 ++
 cpp/src/arrow/python/config.cc               |   35 +
 cpp/src/arrow/python/config.h                |   45 +
 cpp/src/arrow/python/do_import_numpy.h       |   21 +
 cpp/src/arrow/python/helpers.cc              |   55 +
 cpp/src/arrow/python/helpers.h               |   35 +
 cpp/src/arrow/python/io.cc                   |  222 +++
 cpp/src/arrow/python/io.h                    |   99 ++
 cpp/src/arrow/python/numpy_interop.h         |   60 +
 cpp/src/arrow/python/pandas-test.cc          |   64 +
 cpp/src/arrow/python/pandas_convert.cc       | 1936 +++++++++++++++++++++
 cpp/src/arrow/python/pandas_convert.h        |   79 +
 cpp/src/arrow/python/type_traits.h           |  213 +++
 cpp/src/arrow/python/util/CMakeLists.txt     |   39 +
 cpp/src/arrow/python/util/datetime.h         |   42 +
 cpp/src/arrow/python/util/test_main.cc       |   36 +
 python/CMakeLists.txt                        |  215 +--
 python/cmake_modules/FindArrow.cmake         |    9 +
 python/cmake_modules/FindNumPy.cmake         |  100 --
 python/cmake_modules/FindPythonLibsNew.cmake |  241 ---
 python/pyarrow/config.pyx                    |   14 +-
 python/pyarrow/includes/pyarrow.pxd          |    6 +-
 python/setup.py                              |   11 +-
 python/src/pyarrow/CMakeLists.txt            |   22 -
 python/src/pyarrow/adapters/builtin.cc       |  527 ------
 python/src/pyarrow/adapters/builtin.h        |   54 -
 python/src/pyarrow/adapters/pandas-test.cc   |   64 -
 python/src/pyarrow/adapters/pandas.cc        | 1936 ---------------------
 python/src/pyarrow/adapters/pandas.h         |   79 -
 python/src/pyarrow/api.h                     |   26 -
 python/src/pyarrow/common.cc                 |   69 -
 python/src/pyarrow/common.h                  |  137 --
 python/src/pyarrow/config.cc                 |   35 -
 python/src/pyarrow/config.h                  |   46 -
 python/src/pyarrow/do_import_numpy.h         |   21 -
 python/src/pyarrow/helpers.cc                |   55 -
 python/src/pyarrow/helpers.h                 |   35 -
 python/src/pyarrow/io.cc                     |  221 ---
 python/src/pyarrow/io.h                      |   99 --
 python/src/pyarrow/numpy_interop.h           |   60 -
 python/src/pyarrow/type_traits.h             |  212 ---
 python/src/pyarrow/util/CMakeLists.txt       |   39 -
 python/src/pyarrow/util/datetime.h           |   42 -
 python/src/pyarrow/util/test_main.cc         |   36 -
 54 files changed, 4409 insertions(+), 4461 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/ci/travis_script_python.sh
----------------------------------------------------------------------
diff --git a/ci/travis_script_python.sh b/ci/travis_script_python.sh
index 6f4b8e9..df11209 100755
--- a/ci/travis_script_python.sh
+++ b/ci/travis_script_python.sh
@@ -23,7 +23,6 @@ export MINICONDA=$HOME/miniconda
 export PATH="$MINICONDA/bin:$PATH"
 
 export ARROW_HOME=$ARROW_CPP_INSTALL
-export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$ARROW_CPP_INSTALL/lib
 
 pushd $PYTHON_DIR
 export PARQUET_HOME=$TRAVIS_BUILD_DIR/parquet-env
@@ -70,11 +69,31 @@ build_parquet_cpp() {
 
 build_parquet_cpp
 
-export LD_LIBRARY_PATH=$LD_LIBRARY_PATH:$PARQUET_HOME/lib
+function build_arrow_libraries() {
+  CPP_BUILD_DIR=$1
+  CPP_DIR=$TRAVIS_BUILD_DIR/cpp
+
+  mkdir $CPP_BUILD_DIR
+  pushd $CPP_BUILD_DIR
+
+  cmake -DARROW_BUILD_TESTS=off \
+        -DARROW_PYTHON=on \
+        -DCMAKE_INSTALL_PREFIX=$2 \
+        $CPP_DIR
+
+  make -j4
+  make install
+
+  popd
+}
 
 python_version_tests() {
   PYTHON_VERSION=$1
   CONDA_ENV_DIR=$TRAVIS_BUILD_DIR/pyarrow-test-$PYTHON_VERSION
+
+  export ARROW_HOME=$TRAVIS_BUILD_DIR/arrow-install-$PYTHON_VERSION
+  export LD_LIBRARY_PATH=$ARROW_HOME/lib:$PARQUET_HOME/lib
+
   conda create -y -q -p $CONDA_ENV_DIR python=$PYTHON_VERSION
   source activate $CONDA_ENV_DIR
 
@@ -87,6 +106,9 @@ python_version_tests() {
   # Expensive dependencies install from Continuum package repo
   conda install -y pip numpy pandas cython
 
+  # Build C++ libraries
+  build_arrow_libraries arrow-build-$PYTHON_VERSION $ARROW_HOME
+
   # Other stuff pip install
   pip install -r requirements.txt
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/CMakeLists.txt b/cpp/CMakeLists.txt
index c04afe4..c77cf60 100644
--- a/cpp/CMakeLists.txt
+++ b/cpp/CMakeLists.txt
@@ -106,6 +106,10 @@ if("${CMAKE_SOURCE_DIR}" STREQUAL "${CMAKE_CURRENT_SOURCE_DIR}")
     "Rely on boost shared libraries where relevant"
     ON)
 
+  option(ARROW_PYTHON
+    "Build the Arrow CPython extensions"
+    OFF)
+
   option(ARROW_SSE3
     "Build Arrow with SSE3"
     ON)
@@ -133,6 +137,7 @@ if(NOT ARROW_BUILD_BENCHMARKS)
   set(NO_BENCHMARKS 1)
 endif()
 
+include(BuildUtils)
 
 ############################################################
 # Compiler flags
@@ -303,6 +308,14 @@ endfunction()
 #
 # Arguments after the test name will be passed to set_tests_properties().
 function(ADD_ARROW_TEST REL_TEST_NAME)
+  set(options)
+  set(single_value_args)
+  set(multi_value_args STATIC_LINK_LIBS)
+  cmake_parse_arguments(ARG "${options}" "${one_value_args}" "${multi_value_args}" ${ARGN})
+  if(ARG_UNPARSED_ARGUMENTS)
+    message(SEND_ERROR "Error: unrecognized arguments: ${ARG_UNPARSED_ARGUMENTS}")
+  endif()
+
   if(NO_TESTS OR NOT ARROW_BUILD_STATIC)
     return()
   endif()
@@ -312,7 +325,13 @@ function(ADD_ARROW_TEST REL_TEST_NAME)
     # This test has a corresponding .cc file, set it up as an executable.
     set(TEST_PATH "${EXECUTABLE_OUTPUT_PATH}/${TEST_NAME}")
     add_executable(${TEST_NAME} "${REL_TEST_NAME}.cc")
-    target_link_libraries(${TEST_NAME} ${ARROW_TEST_LINK_LIBS})
+
+    if (ARG_STATIC_LINK_LIBS)
+      # Customize link libraries
+      target_link_libraries(${TEST_NAME} ${ARG_STATIC_LINK_LIBS})
+    else()
+      target_link_libraries(${TEST_NAME} ${ARROW_TEST_LINK_LIBS})
+    endif()
     add_dependencies(unittest ${TEST_NAME})
   else()
     # No executable, just invoke the test (probably a script) directly.
@@ -332,10 +351,6 @@ function(ADD_ARROW_TEST REL_TEST_NAME)
       ${BUILD_SUPPORT_DIR}/run-test.sh ${CMAKE_BINARY_DIR} test ${TEST_PATH})
   endif()
   set_tests_properties(${TEST_NAME} PROPERTIES LABELS "unittest")
-
-  if(ARGN)
-    set_tests_properties(${TEST_NAME} PROPERTIES ${ARGN})
-  endif()
 endfunction()
 
 # A wrapper for add_dependencies() that is compatible with NO_TESTS.
@@ -363,72 +378,6 @@ enable_testing()
 ############################################################
 # Dependencies
 ############################################################
-function(ADD_THIRDPARTY_LIB LIB_NAME)
-  set(options)
-  set(one_value_args SHARED_LIB STATIC_LIB)
-  set(multi_value_args DEPS)
-  cmake_parse_arguments(ARG "${options}" "${one_value_args}" "${multi_value_args}" ${ARGN})
-  if(ARG_UNPARSED_ARGUMENTS)
-    message(SEND_ERROR "Error: unrecognized arguments: ${ARG_UNPARSED_ARGUMENTS}")
-  endif()
-
-  if(ARG_STATIC_LIB AND ARG_SHARED_LIB)
-    if(NOT ARG_STATIC_LIB)
-      message(FATAL_ERROR "No static or shared library provided for ${LIB_NAME}")
-    endif()
-
-    SET(AUG_LIB_NAME "${LIB_NAME}_static")
-    add_library(${AUG_LIB_NAME} STATIC IMPORTED)
-    set_target_properties(${AUG_LIB_NAME}
-      PROPERTIES IMPORTED_LOCATION "${ARG_STATIC_LIB}")
-    message("Added static library dependency ${LIB_NAME}: ${ARG_STATIC_LIB}")
-
-    SET(AUG_LIB_NAME "${LIB_NAME}_shared")
-    add_library(${AUG_LIB_NAME} SHARED IMPORTED)
-
-    if(MSVC)
-        # Mark the \u201d.lib\u201d location as part of a Windows DLL
-        set_target_properties(${AUG_LIB_NAME}
-            PROPERTIES IMPORTED_IMPLIB "${ARG_SHARED_LIB}")
-    else()
-        set_target_properties(${AUG_LIB_NAME}
-            PROPERTIES IMPORTED_LOCATION "${ARG_SHARED_LIB}")
-    endif()
-    message("Added shared library dependency ${LIB_NAME}: ${ARG_SHARED_LIB}")
-  elseif(ARG_STATIC_LIB)
-    add_library(${LIB_NAME} STATIC IMPORTED)
-    set_target_properties(${LIB_NAME}
-      PROPERTIES IMPORTED_LOCATION "${ARG_STATIC_LIB}")
-    SET(AUG_LIB_NAME "${LIB_NAME}_static")
-    add_library(${AUG_LIB_NAME} STATIC IMPORTED)
-    set_target_properties(${AUG_LIB_NAME}
-      PROPERTIES IMPORTED_LOCATION "${ARG_STATIC_LIB}")
-    message("Added static library dependency ${LIB_NAME}: ${ARG_STATIC_LIB}")
-  elseif(ARG_SHARED_LIB)
-    add_library(${LIB_NAME} SHARED IMPORTED)
-    set_target_properties(${LIB_NAME}
-      PROPERTIES IMPORTED_LOCATION "${ARG_SHARED_LIB}")
-    SET(AUG_LIB_NAME "${LIB_NAME}_shared")
-    add_library(${AUG_LIB_NAME} SHARED IMPORTED)
-
-    if(MSVC)
-        # Mark the \u201d.lib\u201d location as part of a Windows DLL
-        set_target_properties(${AUG_LIB_NAME}
-            PROPERTIES IMPORTED_IMPLIB "${ARG_SHARED_LIB}")
-    else()
-        set_target_properties(${AUG_LIB_NAME}
-            PROPERTIES IMPORTED_LOCATION "${ARG_SHARED_LIB}")
-    endif()
-    message("Added shared library dependency ${LIB_NAME}: ${ARG_SHARED_LIB}")
-  else()
-    message(FATAL_ERROR "No static or shared library provided for ${LIB_NAME}")
-  endif()
-
-  if(ARG_DEPS)
-    set_target_properties(${LIB_NAME}
-      PROPERTIES IMPORTED_LINK_INTERFACE_LIBRARIES "${ARG_DEPS}")
-  endif()
-endfunction()
 
 # ----------------------------------------------------------------------
 # Add Boost dependencies (code adapted from Apache Kudu (incubating))
@@ -798,8 +747,7 @@ if (${CLANG_FORMAT_FOUND})
   add_custom_target(format ${BUILD_SUPPORT_DIR}/run-clang-format.sh ${CMAKE_CURRENT_SOURCE_DIR} ${CLANG_FORMAT_BIN} 1
     `find ${CMAKE_CURRENT_SOURCE_DIR}/src -name \\*.cc -or -name \\*.h |
     sed -e '/_generated/g' |
-    sed -e '/windows_compatibility.h/g'`
-    `find ${CMAKE_CURRENT_SOURCE_DIR}/../python -name \\*.cc -or -name \\*.h`)
+    sed -e '/windows_compatibility.h/g'`)
 
   # runs clang format and exits with a non-zero exit code if any files need to be reformatted
   add_custom_target(check-format ${BUILD_SUPPORT_DIR}/run-clang-format.sh ${CMAKE_CURRENT_SOURCE_DIR} ${CLANG_FORMAT_BIN} 0
@@ -857,11 +805,9 @@ if(NOT APPLE)
   set(ARROW_SHARED_LINK_FLAGS "-Wl,--version-script=${CMAKE_CURRENT_SOURCE_DIR}/src/arrow/symbols.map")
 endif()
 
-include(BuildUtils)
-
 ADD_ARROW_LIB(arrow
-    SOURCES ${ARROW_SRCS}
-    SHARED_LINK_FLAGS ${ARROW_SHARED_LINK_FLAGS}
+  SOURCES ${ARROW_SRCS}
+  SHARED_LINK_FLAGS ${ARROW_SHARED_LINK_FLAGS}
 )
 
 add_subdirectory(src/arrow)
@@ -875,6 +821,10 @@ endif()
 #----------------------------------------------------------------------
 # IPC library
 
+if(ARROW_PYTHON)
+  set(ARROW_IPC on)
+endif()
+
 ## Flatbuffers
 if(ARROW_IPC)
   if("$ENV{FLATBUFFERS_HOME}" STREQUAL "")
@@ -908,3 +858,14 @@ if(ARROW_IPC)
 
   add_subdirectory(src/arrow/ipc)
 endif()
+
+if(ARROW_PYTHON)
+  find_package(PythonLibsNew REQUIRED)
+  find_package(NumPy REQUIRED)
+
+  include_directories(SYSTEM
+    ${NUMPY_INCLUDE_DIRS}
+    ${PYTHON_INCLUDE_DIRS})
+
+  add_subdirectory(src/arrow/python)
+endif()

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/cmake_modules/BuildUtils.cmake
----------------------------------------------------------------------
diff --git a/cpp/cmake_modules/BuildUtils.cmake b/cpp/cmake_modules/BuildUtils.cmake
index 78b514c..c993041 100644
--- a/cpp/cmake_modules/BuildUtils.cmake
+++ b/cpp/cmake_modules/BuildUtils.cmake
@@ -15,6 +15,73 @@
 # specific language governing permissions and limitations
 # under the License.
 
+function(ADD_THIRDPARTY_LIB LIB_NAME)
+  set(options)
+  set(one_value_args SHARED_LIB STATIC_LIB)
+  set(multi_value_args DEPS)
+  cmake_parse_arguments(ARG "${options}" "${one_value_args}" "${multi_value_args}" ${ARGN})
+  if(ARG_UNPARSED_ARGUMENTS)
+    message(SEND_ERROR "Error: unrecognized arguments: ${ARG_UNPARSED_ARGUMENTS}")
+  endif()
+
+  if(ARG_STATIC_LIB AND ARG_SHARED_LIB)
+    if(NOT ARG_STATIC_LIB)
+      message(FATAL_ERROR "No static or shared library provided for ${LIB_NAME}")
+    endif()
+
+    SET(AUG_LIB_NAME "${LIB_NAME}_static")
+    add_library(${AUG_LIB_NAME} STATIC IMPORTED)
+    set_target_properties(${AUG_LIB_NAME}
+      PROPERTIES IMPORTED_LOCATION "${ARG_STATIC_LIB}")
+    message("Added static library dependency ${LIB_NAME}: ${ARG_STATIC_LIB}")
+
+    SET(AUG_LIB_NAME "${LIB_NAME}_shared")
+    add_library(${AUG_LIB_NAME} SHARED IMPORTED)
+
+    if(MSVC)
+        # Mark the \u201d.lib\u201d location as part of a Windows DLL
+        set_target_properties(${AUG_LIB_NAME}
+            PROPERTIES IMPORTED_IMPLIB "${ARG_SHARED_LIB}")
+    else()
+        set_target_properties(${AUG_LIB_NAME}
+            PROPERTIES IMPORTED_LOCATION "${ARG_SHARED_LIB}")
+    endif()
+    message("Added shared library dependency ${LIB_NAME}: ${ARG_SHARED_LIB}")
+  elseif(ARG_STATIC_LIB)
+    add_library(${LIB_NAME} STATIC IMPORTED)
+    set_target_properties(${LIB_NAME}
+      PROPERTIES IMPORTED_LOCATION "${ARG_STATIC_LIB}")
+    SET(AUG_LIB_NAME "${LIB_NAME}_static")
+    add_library(${AUG_LIB_NAME} STATIC IMPORTED)
+    set_target_properties(${AUG_LIB_NAME}
+      PROPERTIES IMPORTED_LOCATION "${ARG_STATIC_LIB}")
+    message("Added static library dependency ${LIB_NAME}: ${ARG_STATIC_LIB}")
+  elseif(ARG_SHARED_LIB)
+    add_library(${LIB_NAME} SHARED IMPORTED)
+    set_target_properties(${LIB_NAME}
+      PROPERTIES IMPORTED_LOCATION "${ARG_SHARED_LIB}")
+    SET(AUG_LIB_NAME "${LIB_NAME}_shared")
+    add_library(${AUG_LIB_NAME} SHARED IMPORTED)
+
+    if(MSVC)
+        # Mark the \u201d.lib\u201d location as part of a Windows DLL
+        set_target_properties(${AUG_LIB_NAME}
+            PROPERTIES IMPORTED_IMPLIB "${ARG_SHARED_LIB}")
+    else()
+        set_target_properties(${AUG_LIB_NAME}
+            PROPERTIES IMPORTED_LOCATION "${ARG_SHARED_LIB}")
+    endif()
+    message("Added shared library dependency ${LIB_NAME}: ${ARG_SHARED_LIB}")
+  else()
+    message(FATAL_ERROR "No static or shared library provided for ${LIB_NAME}")
+  endif()
+
+  if(ARG_DEPS)
+    set_target_properties(${LIB_NAME}
+      PROPERTIES IMPORTED_LINK_INTERFACE_LIBRARIES "${ARG_DEPS}")
+  endif()
+endfunction()
+
 function(ADD_ARROW_LIB LIB_NAME)
   set(options)
   set(one_value_args SHARED_LINK_FLAGS)
@@ -45,9 +112,16 @@ function(ADD_ARROW_LIB LIB_NAME)
 
   if (ARROW_BUILD_SHARED)
     add_library(${LIB_NAME}_shared SHARED $<TARGET_OBJECTS:${LIB_NAME}_objlib>)
+
     if(APPLE)
-      set_target_properties(${LIB_NAME}_shared PROPERTIES LINK_FLAGS "-undefined dynamic_lookup")
+      # On OS X, you can avoid linking at library load time and instead
+      # expecting that the symbols have been loaded separately. This happens
+      # with libpython* where there can be conflicts between system Python and
+      # the Python from a thirdparty distribution
+      set(ARG_SHARED_LINK_FLAGS
+        "-undefined dynamic_lookup ${ARG_SHARED_LINK_FLAGS}")
     endif()
+
     set_target_properties(${LIB_NAME}_shared
       PROPERTIES
       LIBRARY_OUTPUT_DIRECTORY "${BUILD_OUTPUT_ROOT_DIRECTORY}"
@@ -55,6 +129,7 @@ function(ADD_ARROW_LIB LIB_NAME)
       OUTPUT_NAME ${LIB_NAME}
       VERSION "${ARROW_ABI_VERSION}"
       SOVERSION "${ARROW_SO_VERSION}")
+
     target_link_libraries(${LIB_NAME}_shared
       LINK_PUBLIC ${ARG_SHARED_LINK_LIBS}
       LINK_PRIVATE ${ARG_SHARED_PRIVATE_LINK_LIBS})
@@ -68,28 +143,28 @@ function(ADD_ARROW_LIB LIB_NAME)
         set_target_properties(${LIB_NAME}_shared PROPERTIES
             INSTALL_RPATH ${_lib_install_rpath})
     endif()
-  
+
     install(TARGETS ${LIB_NAME}_shared
       LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR}
       ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR})
   endif()
-  
+
   if (ARROW_BUILD_STATIC)
       add_library(${LIB_NAME}_static STATIC $<TARGET_OBJECTS:${LIB_NAME}_objlib>)
     set_target_properties(${LIB_NAME}_static
       PROPERTIES
       LIBRARY_OUTPUT_DIRECTORY "${BUILD_OUTPUT_ROOT_DIRECTORY}"
       OUTPUT_NAME ${LIB_NAME})
-  
+
   target_link_libraries(${LIB_NAME}_static
       LINK_PUBLIC ${ARG_STATIC_LINK_LIBS}
       LINK_PRIVATE ${ARG_STATIC_PRIVATE_LINK_LIBS})
-  
+
   install(TARGETS ${LIB_NAME}_static
       LIBRARY DESTINATION ${CMAKE_INSTALL_LIBDIR}
       ARCHIVE DESTINATION ${CMAKE_INSTALL_LIBDIR})
   endif()
-  
+
   if (APPLE)
       set_target_properties(${LIB_NAME}_shared
       PROPERTIES
@@ -98,4 +173,3 @@ function(ADD_ARROW_LIB LIB_NAME)
   endif()
 
 endfunction()
-

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/cmake_modules/FindNumPy.cmake
----------------------------------------------------------------------
diff --git a/cpp/cmake_modules/FindNumPy.cmake b/cpp/cmake_modules/FindNumPy.cmake
new file mode 100644
index 0000000..58bb531
--- /dev/null
+++ b/cpp/cmake_modules/FindNumPy.cmake
@@ -0,0 +1,100 @@
+# - Find the NumPy libraries
+# This module finds if NumPy is installed, and sets the following variables
+# indicating where it is.
+#
+# TODO: Update to provide the libraries and paths for linking npymath lib.
+#
+#  NUMPY_FOUND               - was NumPy found
+#  NUMPY_VERSION             - the version of NumPy found as a string
+#  NUMPY_VERSION_MAJOR       - the major version number of NumPy
+#  NUMPY_VERSION_MINOR       - the minor version number of NumPy
+#  NUMPY_VERSION_PATCH       - the patch version number of NumPy
+#  NUMPY_VERSION_DECIMAL     - e.g. version 1.6.1 is 10601
+#  NUMPY_INCLUDE_DIRS        - path to the NumPy include files
+
+#============================================================================
+# Copyright 2012 Continuum Analytics, Inc.
+#
+# MIT License
+#
+# Permission is hereby granted, free of charge, to any person obtaining
+# a copy of this software and associated documentation files
+# (the "Software"), to deal in the Software without restriction, including
+# without limitation the rights to use, copy, modify, merge, publish,
+# distribute, sublicense, and/or sell copies of the Software, and to permit
+# persons to whom the Software is furnished to do so, subject to
+# the following conditions:
+#
+# The above copyright notice and this permission notice shall be included
+# in all copies or substantial portions of the Software.
+#
+# THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS
+# OR IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+# FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL
+# THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR
+# OTHER LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE,
+# ARISING FROM, OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR
+# OTHER DEALINGS IN THE SOFTWARE.
+#
+#============================================================================
+
+# Finding NumPy involves calling the Python interpreter
+if(NumPy_FIND_REQUIRED)
+    find_package(PythonInterp REQUIRED)
+else()
+    find_package(PythonInterp)
+endif()
+
+if(NOT PYTHONINTERP_FOUND)
+    set(NUMPY_FOUND FALSE)
+    return()
+endif()
+
+execute_process(COMMAND "${PYTHON_EXECUTABLE}" "-c"
+    "import numpy as n; print(n.__version__); print(n.get_include());"
+    RESULT_VARIABLE _NUMPY_SEARCH_SUCCESS
+    OUTPUT_VARIABLE _NUMPY_VALUES_OUTPUT
+    ERROR_VARIABLE _NUMPY_ERROR_VALUE
+    OUTPUT_STRIP_TRAILING_WHITESPACE)
+
+if(NOT _NUMPY_SEARCH_SUCCESS MATCHES 0)
+    if(NumPy_FIND_REQUIRED)
+        message(FATAL_ERROR
+            "NumPy import failure:\n${_NUMPY_ERROR_VALUE}")
+    endif()
+    set(NUMPY_FOUND FALSE)
+    return()
+endif()
+
+# Convert the process output into a list
+string(REGEX REPLACE ";" "\\\\;" _NUMPY_VALUES ${_NUMPY_VALUES_OUTPUT})
+string(REGEX REPLACE "\n" ";" _NUMPY_VALUES ${_NUMPY_VALUES})
+list(GET _NUMPY_VALUES 0 NUMPY_VERSION)
+list(GET _NUMPY_VALUES 1 NUMPY_INCLUDE_DIRS)
+
+string(REGEX MATCH "^[0-9]+\\.[0-9]+\\.[0-9]+" _VER_CHECK "${NUMPY_VERSION}")
+if("${_VER_CHECK}" STREQUAL "")
+    # The output from Python was unexpected. Raise an error always
+    # here, because we found NumPy, but it appears to be corrupted somehow.
+    message(FATAL_ERROR
+        "Requested version and include path from NumPy, got instead:\n${_NUMPY_VALUES_OUTPUT}\n")
+    return()
+endif()
+
+# Make sure all directory separators are '/'
+string(REGEX REPLACE "\\\\" "/" NUMPY_INCLUDE_DIRS ${NUMPY_INCLUDE_DIRS})
+
+# Get the major and minor version numbers
+string(REGEX REPLACE "\\." ";" _NUMPY_VERSION_LIST ${NUMPY_VERSION})
+list(GET _NUMPY_VERSION_LIST 0 NUMPY_VERSION_MAJOR)
+list(GET _NUMPY_VERSION_LIST 1 NUMPY_VERSION_MINOR)
+list(GET _NUMPY_VERSION_LIST 2 NUMPY_VERSION_PATCH)
+string(REGEX MATCH "[0-9]*" NUMPY_VERSION_PATCH ${NUMPY_VERSION_PATCH})
+math(EXPR NUMPY_VERSION_DECIMAL
+    "(${NUMPY_VERSION_MAJOR} * 10000) + (${NUMPY_VERSION_MINOR} * 100) + ${NUMPY_VERSION_PATCH}")
+
+find_package_message(NUMPY
+    "Found NumPy: version \"${NUMPY_VERSION}\" ${NUMPY_INCLUDE_DIRS}"
+    "${NUMPY_INCLUDE_DIRS}${NUMPY_VERSION}")
+
+set(NUMPY_FOUND TRUE)

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/cmake_modules/FindPythonLibsNew.cmake
----------------------------------------------------------------------
diff --git a/cpp/cmake_modules/FindPythonLibsNew.cmake b/cpp/cmake_modules/FindPythonLibsNew.cmake
new file mode 100644
index 0000000..1000a95
--- /dev/null
+++ b/cpp/cmake_modules/FindPythonLibsNew.cmake
@@ -0,0 +1,241 @@
+# - Find python libraries
+# This module finds the libraries corresponding to the Python interpeter
+# FindPythonInterp provides.
+# This code sets the following variables:
+#
+#  PYTHONLIBS_FOUND           - have the Python libs been found
+#  PYTHON_PREFIX              - path to the Python installation
+#  PYTHON_LIBRARIES           - path to the python library
+#  PYTHON_INCLUDE_DIRS        - path to where Python.h is found
+#  PYTHON_SITE_PACKAGES       - path to installation site-packages
+#  PYTHON_IS_DEBUG            - whether the Python interpreter is a debug build
+#
+#  PYTHON_INCLUDE_PATH        - path to where Python.h is found (deprecated)
+#
+# A function PYTHON_ADD_MODULE(<name> src1 src2 ... srcN) is defined
+# to build modules for python.
+#
+# Thanks to talljimbo for the patch adding the 'LDVERSION' config
+# variable usage.
+
+#=============================================================================
+# Copyright 2001-2009 Kitware, Inc.
+# Copyright 2012-2014 Continuum Analytics, Inc.
+#
+# All rights reserved.
+#
+# Redistribution and use in source and binary forms, with or without
+# modification, are permitted provided that the following conditions
+# are met:
+#
+# * Redistributions of source code must retain the above copyright
+# notice, this list of conditions and the following disclaimer.
+#
+# * Redistributions in binary form must reproduce the above copyright
+# notice, this list of conditions and the following disclaimer in the
+# documentation and/or other materials provided with the distribution.
+#
+# * Neither the names of Kitware, Inc., the Insight Software Consortium,
+# nor the names of their contributors may be used to endorse or promote
+# products derived from this software without specific prior written
+# permission.
+#
+# THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+# "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+# LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+# # A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+# HOLDER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+# SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+# LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+# DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+# THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+# (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+# OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+#=============================================================================
+# (To distribute this file outside of CMake, substitute the full
+#  License text for the above reference.)
+
+# Use the Python interpreter to find the libs.
+if(PythonLibsNew_FIND_REQUIRED)
+    find_package(PythonInterp REQUIRED)
+else()
+    find_package(PythonInterp)
+endif()
+
+if(NOT PYTHONINTERP_FOUND)
+    set(PYTHONLIBS_FOUND FALSE)
+    return()
+endif()
+
+# According to http://stackoverflow.com/questions/646518/python-how-to-detect-debug-interpreter
+# testing whether sys has the gettotalrefcount function is a reliable,
+# cross-platform way to detect a CPython debug interpreter.
+#
+# The library suffix is from the config var LDVERSION sometimes, otherwise
+# VERSION. VERSION will typically be like "2.7" on unix, and "27" on windows.
+#
+# The config var LIBPL is for Linux, and helps on Debian Jessie where the
+# addition of multi-arch support shuffled things around.
+execute_process(COMMAND "${PYTHON_EXECUTABLE}" "-c"
+    "from distutils import sysconfig as s;import sys;import struct;
+print('.'.join(str(v) for v in sys.version_info));
+print(sys.prefix);
+print(s.get_python_inc(plat_specific=True));
+print(s.get_python_lib(plat_specific=True));
+print(s.get_config_var('SO'));
+print(hasattr(sys, 'gettotalrefcount')+0);
+print(struct.calcsize('@P'));
+print(s.get_config_var('LDVERSION') or s.get_config_var('VERSION'));
+print(s.get_config_var('LIBPL'));
+"
+    RESULT_VARIABLE _PYTHON_SUCCESS
+    OUTPUT_VARIABLE _PYTHON_VALUES
+    ERROR_VARIABLE _PYTHON_ERROR_VALUE
+    OUTPUT_STRIP_TRAILING_WHITESPACE)
+
+if(NOT _PYTHON_SUCCESS MATCHES 0)
+    if(PythonLibsNew_FIND_REQUIRED)
+        message(FATAL_ERROR
+            "Python config failure:\n${_PYTHON_ERROR_VALUE}")
+    endif()
+    set(PYTHONLIBS_FOUND FALSE)
+    return()
+endif()
+
+# Convert the process output into a list
+string(REGEX REPLACE ";" "\\\\;" _PYTHON_VALUES ${_PYTHON_VALUES})
+string(REGEX REPLACE "\n" ";" _PYTHON_VALUES ${_PYTHON_VALUES})
+list(GET _PYTHON_VALUES 0 _PYTHON_VERSION_LIST)
+list(GET _PYTHON_VALUES 1 PYTHON_PREFIX)
+list(GET _PYTHON_VALUES 2 PYTHON_INCLUDE_DIR)
+list(GET _PYTHON_VALUES 3 PYTHON_SITE_PACKAGES)
+list(GET _PYTHON_VALUES 4 PYTHON_MODULE_EXTENSION)
+list(GET _PYTHON_VALUES 5 PYTHON_IS_DEBUG)
+list(GET _PYTHON_VALUES 6 PYTHON_SIZEOF_VOID_P)
+list(GET _PYTHON_VALUES 7 PYTHON_LIBRARY_SUFFIX)
+list(GET _PYTHON_VALUES 8 PYTHON_LIBRARY_PATH)
+
+# Make sure the Python has the same pointer-size as the chosen compiler
+# Skip the check on OS X, it doesn't consistently have CMAKE_SIZEOF_VOID_P defined
+if((NOT APPLE) AND (NOT "${PYTHON_SIZEOF_VOID_P}" STREQUAL "${CMAKE_SIZEOF_VOID_P}"))
+    if(PythonLibsNew_FIND_REQUIRED)
+        math(EXPR _PYTHON_BITS "${PYTHON_SIZEOF_VOID_P} * 8")
+        math(EXPR _CMAKE_BITS "${CMAKE_SIZEOF_VOID_P} * 8")
+        message(FATAL_ERROR
+            "Python config failure: Python is ${_PYTHON_BITS}-bit, "
+            "chosen compiler is  ${_CMAKE_BITS}-bit")
+    endif()
+    set(PYTHONLIBS_FOUND FALSE)
+    return()
+endif()
+
+# The built-in FindPython didn't always give the version numbers
+string(REGEX REPLACE "\\." ";" _PYTHON_VERSION_LIST ${_PYTHON_VERSION_LIST})
+list(GET _PYTHON_VERSION_LIST 0 PYTHON_VERSION_MAJOR)
+list(GET _PYTHON_VERSION_LIST 1 PYTHON_VERSION_MINOR)
+list(GET _PYTHON_VERSION_LIST 2 PYTHON_VERSION_PATCH)
+
+# Make sure all directory separators are '/'
+string(REGEX REPLACE "\\\\" "/" PYTHON_PREFIX ${PYTHON_PREFIX})
+string(REGEX REPLACE "\\\\" "/" PYTHON_INCLUDE_DIR ${PYTHON_INCLUDE_DIR})
+string(REGEX REPLACE "\\\\" "/" PYTHON_SITE_PACKAGES ${PYTHON_SITE_PACKAGES})
+
+if(CMAKE_HOST_WIN32)
+    if("${CMAKE_CXX_COMPILER_ID}" STREQUAL "MSVC")
+        set(PYTHON_LIBRARY
+            "${PYTHON_PREFIX}/libs/Python${PYTHON_LIBRARY_SUFFIX}.lib")
+    else()
+        set(PYTHON_LIBRARY "${PYTHON_PREFIX}/libs/libpython${PYTHON_LIBRARY_SUFFIX}.a")
+    endif()
+elseif(APPLE)
+     # Seems to require "-undefined dynamic_lookup" instead of linking
+     # against the .dylib, otherwise it crashes. This flag is added
+     # below
+    set(PYTHON_LIBRARY "")
+    #set(PYTHON_LIBRARY
+    #    "${PYTHON_PREFIX}/lib/libpython${PYTHON_LIBRARY_SUFFIX}.dylib")
+else()
+    if(${PYTHON_SIZEOF_VOID_P} MATCHES 8)
+        set(_PYTHON_LIBS_SEARCH "${PYTHON_PREFIX}/lib64" "${PYTHON_PREFIX}/lib" "${PYTHON_LIBRARY_PATH}")
+    else()
+        set(_PYTHON_LIBS_SEARCH "${PYTHON_PREFIX}/lib" "${PYTHON_LIBRARY_PATH}")
+    endif()
+    message(STATUS "Searching for Python libs in ${_PYTHON_LIBS_SEARCH}")
+    message(STATUS "Looking for python${PYTHON_LIBRARY_SUFFIX}")
+    # Probably this needs to be more involved. It would be nice if the config
+    # information the python interpreter itself gave us were more complete.
+    find_library(PYTHON_LIBRARY
+        NAMES "python${PYTHON_LIBRARY_SUFFIX}"
+        PATHS ${_PYTHON_LIBS_SEARCH}
+        NO_SYSTEM_ENVIRONMENT_PATH)
+    message(STATUS "Found Python lib ${PYTHON_LIBRARY}")
+endif()
+
+# For backward compatibility, set PYTHON_INCLUDE_PATH, but make it internal.
+SET(PYTHON_INCLUDE_PATH "${PYTHON_INCLUDE_DIR}" CACHE INTERNAL
+          "Path to where Python.h is found (deprecated)")
+
+MARK_AS_ADVANCED(
+  PYTHON_LIBRARY
+  PYTHON_INCLUDE_DIR
+)
+
+# We use PYTHON_INCLUDE_DIR, PYTHON_LIBRARY and PYTHON_DEBUG_LIBRARY for the
+# cache entries because they are meant to specify the location of a single
+# library. We now set the variables listed by the documentation for this
+# module.
+SET(PYTHON_INCLUDE_DIRS "${PYTHON_INCLUDE_DIR}")
+SET(PYTHON_LIBRARIES "${PYTHON_LIBRARY}")
+SET(PYTHON_DEBUG_LIBRARIES "${PYTHON_DEBUG_LIBRARY}")
+
+
+# Don't know how to get to this directory, just doing something simple :P
+#INCLUDE(${CMAKE_CURRENT_LIST_DIR}/FindPackageHandleStandardArgs.cmake)
+#FIND_PACKAGE_HANDLE_STANDARD_ARGS(PythonLibs DEFAULT_MSG PYTHON_LIBRARIES PYTHON_INCLUDE_DIRS)
+find_package_message(PYTHON
+    "Found PythonLibs: ${PYTHON_LIBRARY}"
+    "${PYTHON_EXECUTABLE}${PYTHON_VERSION}")
+
+
+# PYTHON_ADD_MODULE(<name> src1 src2 ... srcN) is used to build modules for python.
+FUNCTION(PYTHON_ADD_MODULE _NAME )
+  GET_PROPERTY(_TARGET_SUPPORTS_SHARED_LIBS
+    GLOBAL PROPERTY TARGET_SUPPORTS_SHARED_LIBS)
+  OPTION(PYTHON_ENABLE_MODULE_${_NAME} "Add module ${_NAME}" TRUE)
+  OPTION(PYTHON_MODULE_${_NAME}_BUILD_SHARED
+    "Add module ${_NAME} shared" ${_TARGET_SUPPORTS_SHARED_LIBS})
+
+  # Mark these options as advanced
+  MARK_AS_ADVANCED(PYTHON_ENABLE_MODULE_${_NAME}
+    PYTHON_MODULE_${_NAME}_BUILD_SHARED)
+
+  IF(PYTHON_ENABLE_MODULE_${_NAME})
+    IF(PYTHON_MODULE_${_NAME}_BUILD_SHARED)
+      SET(PY_MODULE_TYPE MODULE)
+    ELSE(PYTHON_MODULE_${_NAME}_BUILD_SHARED)
+      SET(PY_MODULE_TYPE STATIC)
+      SET_PROPERTY(GLOBAL  APPEND  PROPERTY  PY_STATIC_MODULES_LIST ${_NAME})
+    ENDIF(PYTHON_MODULE_${_NAME}_BUILD_SHARED)
+
+    SET_PROPERTY(GLOBAL  APPEND  PROPERTY  PY_MODULES_LIST ${_NAME})
+    ADD_LIBRARY(${_NAME} ${PY_MODULE_TYPE} ${ARGN})
+    IF(APPLE)
+      # On OS X, linking against the Python libraries causes
+      # segfaults, so do this dynamic lookup instead.
+      SET_TARGET_PROPERTIES(${_NAME} PROPERTIES LINK_FLAGS
+                          "-undefined dynamic_lookup")
+    ELSE()
+     # In general, we should not link against libpython as we do not embed
+     # the Python interpreter. The python binary itself can then define where
+     # the symbols should loaded from.
+     SET_TARGET_PROPERTIES(${_NAME} PROPERTIES LINK_FLAGS
+         "-Wl,-undefined,dynamic_lookup")
+    ENDIF()
+    IF(PYTHON_MODULE_${_NAME}_BUILD_SHARED)
+      SET_TARGET_PROPERTIES(${_NAME} PROPERTIES PREFIX "${PYTHON_MODULE_PREFIX}")
+      SET_TARGET_PROPERTIES(${_NAME} PROPERTIES SUFFIX "${PYTHON_MODULE_EXTENSION}")
+    ELSE()
+    ENDIF()
+
+  ENDIF(PYTHON_ENABLE_MODULE_${_NAME})
+ENDFUNCTION(PYTHON_ADD_MODULE)

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/CMakeLists.txt b/cpp/src/arrow/python/CMakeLists.txt
new file mode 100644
index 0000000..03f5afc
--- /dev/null
+++ b/cpp/src/arrow/python/CMakeLists.txt
@@ -0,0 +1,93 @@
+# 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
+#
+
+#######################################
+# arrow_python
+#######################################
+
+if (ARROW_BUILD_TESTS)
+  add_library(arrow_python_test_main STATIC
+	util/test_main.cc)
+
+  if (APPLE)
+	target_link_libraries(arrow_python_test_main
+      gtest
+      dl)
+	set_target_properties(arrow_python_test_main
+      PROPERTIES LINK_FLAGS "-undefined dynamic_lookup")
+  else()
+	target_link_libraries(arrow_python_test_main
+      gtest
+      pthread
+      dl
+	  )
+  endif()
+endif()
+
+set(ARROW_PYTHON_MIN_TEST_LIBS
+  arrow_python_test_main
+  arrow_python_static
+  arrow_ipc_static
+  arrow_io_static
+  arrow_static)
+
+if(NOT APPLE AND ARROW_BUILD_TESTS)
+  ADD_THIRDPARTY_LIB(python
+    SHARED_LIB "${PYTHON_LIBRARIES}")
+  list(APPEND ARROW_PYTHON_MIN_TEST_LIBS python)
+endif()
+
+set(ARROW_PYTHON_TEST_LINK_LIBS ${ARROW_PYTHON_MIN_TEST_LIBS})
+
+# ----------------------------------------------------------------------
+
+set(ARROW_PYTHON_SRCS
+  builtin_convert.cc
+  common.cc
+  config.cc
+  helpers.cc
+  io.cc
+  pandas_convert.cc
+)
+
+set(ARROW_PYTHON_SHARED_LINK_LIBS
+  arrow_io_shared
+  arrow_ipc_shared
+  arrow_shared
+)
+
+ADD_ARROW_LIB(arrow_python
+  SOURCES ${ARROW_PYTHON_SRCS}
+  SHARED_LINK_FLAGS ""
+  SHARED_LINK_LIBS ${ARROW_PYTHON_SHARED_LINK_LIBS}
+  STATIC_LINK_LIBS ${ARROW_IO_SHARED_PRIVATE_LINK_LIBS}
+)
+
+install(FILES
+  api.h
+  builtin_convert.h
+  common.h
+  config.h
+  do_import_numpy.h
+  helpers.h
+  io.h
+  numpy_interop.h
+  pandas_convert.h
+  type_traits.h
+  DESTINATION "${CMAKE_INSTALL_INCLUDEDIR}/arrow/python")
+
+# set_target_properties(arrow_python_shared PROPERTIES
+#     INSTALL_RPATH "\$ORIGIN")
+
+if (ARROW_BUILD_TESTS)
+  ADD_ARROW_TEST(pandas-test
+    STATIC_LINK_LIBS "${ARROW_PYTHON_TEST_LINK_LIBS}")
+endif()

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/api.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/api.h b/cpp/src/arrow/python/api.h
new file mode 100644
index 0000000..f4f1c0c
--- /dev/null
+++ b/cpp/src/arrow/python/api.h
@@ -0,0 +1,27 @@
+// 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.
+
+#ifndef ARROW_PYTHON_API_H
+#define ARROW_PYTHON_API_H
+
+#include "arrow/python/builtin_convert.h"
+#include "arrow/python/common.h"
+#include "arrow/python/helpers.h"
+#include "arrow/python/io.h"
+#include "arrow/python/pandas_convert.h"
+
+#endif  // ARROW_PYTHON_API_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/builtin_convert.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/builtin_convert.cc b/cpp/src/arrow/python/builtin_convert.cc
new file mode 100644
index 0000000..9acccc1
--- /dev/null
+++ b/cpp/src/arrow/python/builtin_convert.cc
@@ -0,0 +1,527 @@
+// 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 <Python.h>
+#include <datetime.h>
+#include <sstream>
+
+#include "arrow/python/builtin_convert.h"
+
+#include "arrow/api.h"
+#include "arrow/status.h"
+
+#include "arrow/python/helpers.h"
+#include "arrow/python/util/datetime.h"
+
+namespace arrow {
+namespace py {
+
+static inline bool IsPyInteger(PyObject* obj) {
+#if PYARROW_IS_PY2
+  return PyLong_Check(obj) || PyInt_Check(obj);
+#else
+  return PyLong_Check(obj);
+#endif
+}
+
+class ScalarVisitor {
+ public:
+  ScalarVisitor()
+      : total_count_(0),
+        none_count_(0),
+        bool_count_(0),
+        int_count_(0),
+        date_count_(0),
+        timestamp_count_(0),
+        float_count_(0),
+        binary_count_(0),
+        unicode_count_(0) {}
+
+  void Visit(PyObject* obj) {
+    ++total_count_;
+    if (obj == Py_None) {
+      ++none_count_;
+    } else if (PyBool_Check(obj)) {
+      ++bool_count_;
+    } else if (PyFloat_Check(obj)) {
+      ++float_count_;
+    } else if (IsPyInteger(obj)) {
+      ++int_count_;
+    } else if (PyDate_CheckExact(obj)) {
+      ++date_count_;
+    } else if (PyDateTime_CheckExact(obj)) {
+      ++timestamp_count_;
+    } else if (PyBytes_Check(obj)) {
+      ++binary_count_;
+    } else if (PyUnicode_Check(obj)) {
+      ++unicode_count_;
+    } else {
+      // TODO(wesm): accumulate error information somewhere
+    }
+  }
+
+  std::shared_ptr<DataType> GetType() {
+    // TODO(wesm): handling mixed-type cases
+    if (float_count_) {
+      return float64();
+    } else if (int_count_) {
+      // TODO(wesm): tighter type later
+      return int64();
+    } else if (date_count_) {
+      return date64();
+    } else if (timestamp_count_) {
+      return timestamp(TimeUnit::MICRO);
+    } else if (bool_count_) {
+      return boolean();
+    } else if (binary_count_) {
+      return binary();
+    } else if (unicode_count_) {
+      return utf8();
+    } else {
+      return null();
+    }
+  }
+
+  int64_t total_count() const { return total_count_; }
+
+ private:
+  int64_t total_count_;
+  int64_t none_count_;
+  int64_t bool_count_;
+  int64_t int_count_;
+  int64_t date_count_;
+  int64_t timestamp_count_;
+  int64_t float_count_;
+  int64_t binary_count_;
+  int64_t unicode_count_;
+
+  // Place to accumulate errors
+  // std::vector<Status> errors_;
+};
+
+static constexpr int MAX_NESTING_LEVELS = 32;
+
+class SeqVisitor {
+ public:
+  SeqVisitor() : max_nesting_level_(0) {
+    memset(nesting_histogram_, 0, MAX_NESTING_LEVELS * sizeof(int));
+  }
+
+  Status Visit(PyObject* obj, int level = 0) {
+    Py_ssize_t size = PySequence_Size(obj);
+
+    if (level > max_nesting_level_) { max_nesting_level_ = level; }
+
+    for (int64_t i = 0; i < size; ++i) {
+      // TODO(wesm): Error checking?
+      // TODO(wesm): Specialize for PyList_GET_ITEM?
+      OwnedRef item_ref(PySequence_GetItem(obj, i));
+      PyObject* item = item_ref.obj();
+
+      if (PyList_Check(item)) {
+        RETURN_NOT_OK(Visit(item, level + 1));
+      } else if (PyDict_Check(item)) {
+        return Status::NotImplemented("No type inference for dicts");
+      } else {
+        // We permit nulls at any level of nesting
+        if (item == Py_None) {
+          // TODO
+        } else {
+          ++nesting_histogram_[level];
+          scalars_.Visit(item);
+        }
+      }
+    }
+    return Status::OK();
+  }
+
+  std::shared_ptr<DataType> GetType() {
+    if (scalars_.total_count() == 0) {
+      if (max_nesting_level_ == 0) {
+        return null();
+      } else {
+        return nullptr;
+      }
+    } else {
+      std::shared_ptr<DataType> result = scalars_.GetType();
+      for (int i = 0; i < max_nesting_level_; ++i) {
+        result = std::make_shared<ListType>(result);
+      }
+      return result;
+    }
+  }
+
+  Status Validate() const {
+    if (scalars_.total_count() > 0) {
+      if (num_nesting_levels() > 1) {
+        return Status::Invalid("Mixed nesting levels not supported");
+      } else if (max_observed_level() < max_nesting_level_) {
+        return Status::Invalid("Mixed nesting levels not supported");
+      }
+    }
+    return Status::OK();
+  }
+
+  int max_observed_level() const {
+    int result = 0;
+    for (int i = 0; i < MAX_NESTING_LEVELS; ++i) {
+      if (nesting_histogram_[i] > 0) { result = i; }
+    }
+    return result;
+  }
+
+  int num_nesting_levels() const {
+    int result = 0;
+    for (int i = 0; i < MAX_NESTING_LEVELS; ++i) {
+      if (nesting_histogram_[i] > 0) { ++result; }
+    }
+    return result;
+  }
+
+ private:
+  ScalarVisitor scalars_;
+
+  // Track observed
+  int max_nesting_level_;
+  int nesting_histogram_[MAX_NESTING_LEVELS];
+};
+
+// Non-exhaustive type inference
+Status InferArrowType(PyObject* obj, int64_t* size, std::shared_ptr<DataType>* out_type) {
+  *size = PySequence_Size(obj);
+  if (PyErr_Occurred()) {
+    // Not a sequence
+    PyErr_Clear();
+    return Status::TypeError("Object is not a sequence");
+  }
+
+  // For 0-length sequences, refuse to guess
+  if (*size == 0) { *out_type = null(); }
+
+  SeqVisitor seq_visitor;
+  RETURN_NOT_OK(seq_visitor.Visit(obj));
+  RETURN_NOT_OK(seq_visitor.Validate());
+
+  *out_type = seq_visitor.GetType();
+
+  if (*out_type == nullptr) { return Status::TypeError("Unable to determine data type"); }
+
+  return Status::OK();
+}
+
+// Marshal Python sequence (list, tuple, etc.) to Arrow array
+class SeqConverter {
+ public:
+  virtual Status Init(const std::shared_ptr<ArrayBuilder>& builder) {
+    builder_ = builder;
+    return Status::OK();
+  }
+
+  virtual Status AppendData(PyObject* seq) = 0;
+
+ protected:
+  std::shared_ptr<ArrayBuilder> builder_;
+};
+
+template <typename BuilderType>
+class TypedConverter : public SeqConverter {
+ public:
+  Status Init(const std::shared_ptr<ArrayBuilder>& builder) override {
+    builder_ = builder;
+    typed_builder_ = static_cast<BuilderType*>(builder.get());
+    return Status::OK();
+  }
+
+ protected:
+  BuilderType* typed_builder_;
+};
+
+class BoolConverter : public TypedConverter<BooleanBuilder> {
+ public:
+  Status AppendData(PyObject* seq) override {
+    Py_ssize_t size = PySequence_Size(seq);
+    RETURN_NOT_OK(typed_builder_->Reserve(size));
+    for (int64_t i = 0; i < size; ++i) {
+      OwnedRef item(PySequence_GetItem(seq, i));
+      if (item.obj() == Py_None) {
+        typed_builder_->AppendNull();
+      } else {
+        if (item.obj() == Py_True) {
+          typed_builder_->Append(true);
+        } else {
+          typed_builder_->Append(false);
+        }
+      }
+    }
+    return Status::OK();
+  }
+};
+
+class Int64Converter : public TypedConverter<Int64Builder> {
+ public:
+  Status AppendData(PyObject* seq) override {
+    int64_t val;
+    Py_ssize_t size = PySequence_Size(seq);
+    RETURN_NOT_OK(typed_builder_->Reserve(size));
+    for (int64_t i = 0; i < size; ++i) {
+      OwnedRef item(PySequence_GetItem(seq, i));
+      if (item.obj() == Py_None) {
+        typed_builder_->AppendNull();
+      } else {
+        val = PyLong_AsLongLong(item.obj());
+        RETURN_IF_PYERROR();
+        typed_builder_->Append(val);
+      }
+    }
+    return Status::OK();
+  }
+};
+
+class DateConverter : public TypedConverter<Date64Builder> {
+ public:
+  Status AppendData(PyObject* seq) override {
+    Py_ssize_t size = PySequence_Size(seq);
+    RETURN_NOT_OK(typed_builder_->Reserve(size));
+    for (int64_t i = 0; i < size; ++i) {
+      OwnedRef item(PySequence_GetItem(seq, i));
+      if (item.obj() == Py_None) {
+        typed_builder_->AppendNull();
+      } else {
+        PyDateTime_Date* pydate = reinterpret_cast<PyDateTime_Date*>(item.obj());
+        typed_builder_->Append(PyDate_to_ms(pydate));
+      }
+    }
+    return Status::OK();
+  }
+};
+
+class TimestampConverter : public TypedConverter<TimestampBuilder> {
+ public:
+  Status AppendData(PyObject* seq) override {
+    Py_ssize_t size = PySequence_Size(seq);
+    RETURN_NOT_OK(typed_builder_->Reserve(size));
+    for (int64_t i = 0; i < size; ++i) {
+      OwnedRef item(PySequence_GetItem(seq, i));
+      if (item.obj() == Py_None) {
+        typed_builder_->AppendNull();
+      } else {
+        PyDateTime_DateTime* pydatetime =
+            reinterpret_cast<PyDateTime_DateTime*>(item.obj());
+        struct tm datetime = {0};
+        datetime.tm_year = PyDateTime_GET_YEAR(pydatetime) - 1900;
+        datetime.tm_mon = PyDateTime_GET_MONTH(pydatetime) - 1;
+        datetime.tm_mday = PyDateTime_GET_DAY(pydatetime);
+        datetime.tm_hour = PyDateTime_DATE_GET_HOUR(pydatetime);
+        datetime.tm_min = PyDateTime_DATE_GET_MINUTE(pydatetime);
+        datetime.tm_sec = PyDateTime_DATE_GET_SECOND(pydatetime);
+        int us = PyDateTime_DATE_GET_MICROSECOND(pydatetime);
+        RETURN_IF_PYERROR();
+        struct tm epoch = {0};
+        epoch.tm_year = 70;
+        epoch.tm_mday = 1;
+        // Microseconds since the epoch
+        int64_t val = lrint(difftime(mktime(&datetime), mktime(&epoch))) * 1000000 + us;
+        typed_builder_->Append(val);
+      }
+    }
+    return Status::OK();
+  }
+};
+
+class DoubleConverter : public TypedConverter<DoubleBuilder> {
+ public:
+  Status AppendData(PyObject* seq) override {
+    double val;
+    Py_ssize_t size = PySequence_Size(seq);
+    RETURN_NOT_OK(typed_builder_->Reserve(size));
+    for (int64_t i = 0; i < size; ++i) {
+      OwnedRef item(PySequence_GetItem(seq, i));
+      if (item.obj() == Py_None) {
+        typed_builder_->AppendNull();
+      } else {
+        val = PyFloat_AsDouble(item.obj());
+        RETURN_IF_PYERROR();
+        typed_builder_->Append(val);
+      }
+    }
+    return Status::OK();
+  }
+};
+
+class BytesConverter : public TypedConverter<BinaryBuilder> {
+ public:
+  Status AppendData(PyObject* seq) override {
+    PyObject* item;
+    PyObject* bytes_obj;
+    OwnedRef tmp;
+    const char* bytes;
+    int64_t length;
+    Py_ssize_t size = PySequence_Size(seq);
+    for (int64_t i = 0; i < size; ++i) {
+      item = PySequence_GetItem(seq, i);
+      OwnedRef holder(item);
+
+      if (item == Py_None) {
+        RETURN_NOT_OK(typed_builder_->AppendNull());
+        continue;
+      } else if (PyUnicode_Check(item)) {
+        tmp.reset(PyUnicode_AsUTF8String(item));
+        RETURN_IF_PYERROR();
+        bytes_obj = tmp.obj();
+      } else if (PyBytes_Check(item)) {
+        bytes_obj = item;
+      } else {
+        return Status::TypeError("Non-string value encountered");
+      }
+      // No error checking
+      length = PyBytes_GET_SIZE(bytes_obj);
+      bytes = PyBytes_AS_STRING(bytes_obj);
+      RETURN_NOT_OK(typed_builder_->Append(bytes, length));
+    }
+    return Status::OK();
+  }
+};
+
+class UTF8Converter : public TypedConverter<StringBuilder> {
+ public:
+  Status AppendData(PyObject* seq) override {
+    PyObject* item;
+    PyObject* bytes_obj;
+    OwnedRef tmp;
+    const char* bytes;
+    int64_t length;
+    Py_ssize_t size = PySequence_Size(seq);
+    for (int64_t i = 0; i < size; ++i) {
+      item = PySequence_GetItem(seq, i);
+      OwnedRef holder(item);
+
+      if (item == Py_None) {
+        RETURN_NOT_OK(typed_builder_->AppendNull());
+        continue;
+      } else if (!PyUnicode_Check(item)) {
+        return Status::TypeError("Non-unicode value encountered");
+      }
+      tmp.reset(PyUnicode_AsUTF8String(item));
+      RETURN_IF_PYERROR();
+      bytes_obj = tmp.obj();
+
+      // No error checking
+      length = PyBytes_GET_SIZE(bytes_obj);
+      bytes = PyBytes_AS_STRING(bytes_obj);
+      RETURN_NOT_OK(typed_builder_->Append(bytes, length));
+    }
+    return Status::OK();
+  }
+};
+
+class ListConverter : public TypedConverter<ListBuilder> {
+ public:
+  Status Init(const std::shared_ptr<ArrayBuilder>& builder) override;
+
+  Status AppendData(PyObject* seq) override {
+    Py_ssize_t size = PySequence_Size(seq);
+    for (int64_t i = 0; i < size; ++i) {
+      OwnedRef item(PySequence_GetItem(seq, i));
+      if (item.obj() == Py_None) {
+        RETURN_NOT_OK(typed_builder_->AppendNull());
+      } else {
+        typed_builder_->Append();
+        RETURN_NOT_OK(value_converter_->AppendData(item.obj()));
+      }
+    }
+    return Status::OK();
+  }
+
+ protected:
+  std::shared_ptr<SeqConverter> value_converter_;
+};
+
+// Dynamic constructor for sequence converters
+std::shared_ptr<SeqConverter> GetConverter(const std::shared_ptr<DataType>& type) {
+  switch (type->type) {
+    case Type::BOOL:
+      return std::make_shared<BoolConverter>();
+    case Type::INT64:
+      return std::make_shared<Int64Converter>();
+    case Type::DATE64:
+      return std::make_shared<DateConverter>();
+    case Type::TIMESTAMP:
+      return std::make_shared<TimestampConverter>();
+    case Type::DOUBLE:
+      return std::make_shared<DoubleConverter>();
+    case Type::BINARY:
+      return std::make_shared<BytesConverter>();
+    case Type::STRING:
+      return std::make_shared<UTF8Converter>();
+    case Type::LIST:
+      return std::make_shared<ListConverter>();
+    case Type::STRUCT:
+    default:
+      return nullptr;
+      break;
+  }
+}
+
+Status ListConverter::Init(const std::shared_ptr<ArrayBuilder>& builder) {
+  builder_ = builder;
+  typed_builder_ = static_cast<ListBuilder*>(builder.get());
+
+  value_converter_ =
+      GetConverter(static_cast<ListType*>(builder->type().get())->value_type());
+  if (value_converter_ == nullptr) {
+    return Status::NotImplemented("value type not implemented");
+  }
+
+  value_converter_->Init(typed_builder_->value_builder());
+  return Status::OK();
+}
+
+Status AppendPySequence(PyObject* obj, const std::shared_ptr<DataType>& type,
+    const std::shared_ptr<ArrayBuilder>& builder) {
+  std::shared_ptr<SeqConverter> converter = GetConverter(type);
+  if (converter == nullptr) {
+    std::stringstream ss;
+    ss << "No type converter implemented for " << type->ToString();
+    return Status::NotImplemented(ss.str());
+  }
+  converter->Init(builder);
+
+  return converter->AppendData(obj);
+}
+
+Status ConvertPySequence(PyObject* obj, MemoryPool* pool, std::shared_ptr<Array>* out) {
+  std::shared_ptr<DataType> type;
+  int64_t size;
+  PyDateTime_IMPORT;
+  RETURN_NOT_OK(InferArrowType(obj, &size, &type));
+
+  // Handle NA / NullType case
+  if (type->type == Type::NA) {
+    out->reset(new NullArray(size));
+    return Status::OK();
+  }
+
+  // Give the sequence converter an array builder
+  std::shared_ptr<ArrayBuilder> builder;
+  RETURN_NOT_OK(MakeBuilder(pool, type, &builder));
+  RETURN_NOT_OK(AppendPySequence(obj, type, builder));
+
+  return builder->Finish(out);
+}
+
+}  // namespace py
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/builtin_convert.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/builtin_convert.h b/cpp/src/arrow/python/builtin_convert.h
new file mode 100644
index 0000000..7b50990
--- /dev/null
+++ b/cpp/src/arrow/python/builtin_convert.h
@@ -0,0 +1,54 @@
+// 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.
+
+// Functions for converting between CPython built-in data structures and Arrow
+// data structures
+
+#ifndef ARROW_PYTHON_ADAPTERS_BUILTIN_H
+#define ARROW_PYTHON_ADAPTERS_BUILTIN_H
+
+#include <Python.h>
+
+#include <memory>
+
+#include <arrow/type.h>
+
+#include "arrow/util/visibility.h"
+
+#include "arrow/python/common.h"
+
+namespace arrow {
+
+class Array;
+class Status;
+
+namespace py {
+
+ARROW_EXPORT arrow::Status InferArrowType(
+    PyObject* obj, int64_t* size, std::shared_ptr<arrow::DataType>* out_type);
+
+ARROW_EXPORT arrow::Status AppendPySequence(PyObject* obj,
+    const std::shared_ptr<arrow::DataType>& type,
+    const std::shared_ptr<arrow::ArrayBuilder>& builder);
+
+ARROW_EXPORT
+Status ConvertPySequence(PyObject* obj, MemoryPool* pool, std::shared_ptr<Array>* out);
+
+}  // namespace py
+}  // namespace arrow
+
+#endif  // ARROW_PYTHON_ADAPTERS_BUILTIN_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/common.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/common.cc b/cpp/src/arrow/python/common.cc
new file mode 100644
index 0000000..a5aea30
--- /dev/null
+++ b/cpp/src/arrow/python/common.cc
@@ -0,0 +1,68 @@
+// 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 "arrow/python/common.h"
+
+#include <cstdlib>
+#include <mutex>
+#include <sstream>
+
+#include "arrow/memory_pool.h"
+#include "arrow/status.h"
+
+namespace arrow {
+namespace py {
+
+static std::mutex memory_pool_mutex;
+static MemoryPool* default_python_pool = nullptr;
+
+void set_default_memory_pool(MemoryPool* pool) {
+  std::lock_guard<std::mutex> guard(memory_pool_mutex);
+  default_python_pool = pool;
+}
+
+MemoryPool* get_memory_pool() {
+  std::lock_guard<std::mutex> guard(memory_pool_mutex);
+  if (default_python_pool) {
+    return default_python_pool;
+  } else {
+    return default_memory_pool();
+  }
+}
+
+// ----------------------------------------------------------------------
+// PyBuffer
+
+PyBuffer::PyBuffer(PyObject* obj) : Buffer(nullptr, 0) {
+  if (PyObject_CheckBuffer(obj)) {
+    obj_ = PyMemoryView_FromObject(obj);
+    Py_buffer* buffer = PyMemoryView_GET_BUFFER(obj_);
+    data_ = reinterpret_cast<const uint8_t*>(buffer->buf);
+    size_ = buffer->len;
+    capacity_ = buffer->len;
+    is_mutable_ = false;
+    Py_INCREF(obj_);
+  }
+}
+
+PyBuffer::~PyBuffer() {
+  PyAcquireGIL lock;
+  Py_DECREF(obj_);
+}
+
+}  // namespace py
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/common.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/common.h b/cpp/src/arrow/python/common.h
new file mode 100644
index 0000000..f1be471
--- /dev/null
+++ b/cpp/src/arrow/python/common.h
@@ -0,0 +1,139 @@
+// 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.
+
+#ifndef ARROW_PYTHON_COMMON_H
+#define ARROW_PYTHON_COMMON_H
+
+#include <string>
+
+#include "arrow/python/config.h"
+
+#include "arrow/buffer.h"
+#include "arrow/util/macros.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+
+class MemoryPool;
+
+namespace py {
+
+class PyAcquireGIL {
+ public:
+  PyAcquireGIL() { state_ = PyGILState_Ensure(); }
+
+  ~PyAcquireGIL() { PyGILState_Release(state_); }
+
+ private:
+  PyGILState_STATE state_;
+  DISALLOW_COPY_AND_ASSIGN(PyAcquireGIL);
+};
+
+#define PYARROW_IS_PY2 PY_MAJOR_VERSION <= 2
+
+class OwnedRef {
+ public:
+  OwnedRef() : obj_(nullptr) {}
+
+  explicit OwnedRef(PyObject* obj) : obj_(obj) {}
+
+  ~OwnedRef() {
+    PyAcquireGIL lock;
+    Py_XDECREF(obj_);
+  }
+
+  void reset(PyObject* obj) {
+    if (obj_ != nullptr) { Py_XDECREF(obj_); }
+    obj_ = obj;
+  }
+
+  void release() { obj_ = nullptr; }
+
+  PyObject* obj() const { return obj_; }
+
+ private:
+  PyObject* obj_;
+};
+
+struct PyObjectStringify {
+  OwnedRef tmp_obj;
+  const char* bytes;
+
+  explicit PyObjectStringify(PyObject* obj) {
+    PyObject* bytes_obj;
+    if (PyUnicode_Check(obj)) {
+      bytes_obj = PyUnicode_AsUTF8String(obj);
+      tmp_obj.reset(bytes_obj);
+    } else {
+      bytes_obj = obj;
+    }
+    bytes = PyBytes_AsString(bytes_obj);
+  }
+};
+
+// TODO(wesm): We can just let errors pass through. To be explored later
+#define RETURN_IF_PYERROR()                         \
+  if (PyErr_Occurred()) {                           \
+    PyObject *exc_type, *exc_value, *traceback;     \
+    PyErr_Fetch(&exc_type, &exc_value, &traceback); \
+    PyObjectStringify stringified(exc_value);       \
+    std::string message(stringified.bytes);         \
+    Py_DECREF(exc_type);                            \
+    Py_XDECREF(exc_value);                          \
+    Py_XDECREF(traceback);                          \
+    PyErr_Clear();                                  \
+    return Status::UnknownError(message);           \
+  }
+
+// Return the common PyArrow memory pool
+ARROW_EXPORT void set_default_memory_pool(MemoryPool* pool);
+ARROW_EXPORT MemoryPool* get_memory_pool();
+
+class ARROW_EXPORT NumPyBuffer : public Buffer {
+ public:
+  explicit NumPyBuffer(PyArrayObject* arr) : Buffer(nullptr, 0) {
+    arr_ = arr;
+    Py_INCREF(arr);
+
+    data_ = reinterpret_cast<const uint8_t*>(PyArray_DATA(arr_));
+    size_ = PyArray_SIZE(arr_) * PyArray_DESCR(arr_)->elsize;
+    capacity_ = size_;
+  }
+
+  virtual ~NumPyBuffer() { Py_XDECREF(arr_); }
+
+ private:
+  PyArrayObject* arr_;
+};
+
+class ARROW_EXPORT PyBuffer : public Buffer {
+ public:
+  /// Note that the GIL must be held when calling the PyBuffer constructor.
+  ///
+  /// While memoryview objects support multi-demensional buffers, PyBuffer only supports
+  /// one-dimensional byte buffers.
+  explicit PyBuffer(PyObject* obj);
+  ~PyBuffer();
+
+ private:
+  PyObject* obj_;
+};
+
+}  // namespace py
+}  // namespace arrow
+
+#endif  // ARROW_PYTHON_COMMON_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/config.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/config.cc b/cpp/src/arrow/python/config.cc
new file mode 100644
index 0000000..2abc4dd
--- /dev/null
+++ b/cpp/src/arrow/python/config.cc
@@ -0,0 +1,35 @@
+// 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 <Python.h>
+
+#include "arrow/python/config.h"
+
+namespace arrow {
+namespace py {
+
+void Init() {}
+
+PyObject* numpy_nan = nullptr;
+
+void set_numpy_nan(PyObject* obj) {
+  Py_INCREF(obj);
+  numpy_nan = obj;
+}
+
+}  // namespace py
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/config.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/config.h b/cpp/src/arrow/python/config.h
new file mode 100644
index 0000000..dd554e0
--- /dev/null
+++ b/cpp/src/arrow/python/config.h
@@ -0,0 +1,45 @@
+// 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.
+
+#ifndef ARROW_PYTHON_CONFIG_H
+#define ARROW_PYTHON_CONFIG_H
+
+#include <Python.h>
+
+#include "arrow/python/numpy_interop.h"
+#include "arrow/util/visibility.h"
+
+#if PY_MAJOR_VERSION >= 3
+#define PyString_Check PyUnicode_Check
+#endif
+
+namespace arrow {
+namespace py {
+
+ARROW_EXPORT
+extern PyObject* numpy_nan;
+
+ARROW_EXPORT
+void Init();
+
+ARROW_EXPORT
+void set_numpy_nan(PyObject* obj);
+
+}  // namespace py
+}  // namespace arrow
+
+#endif  // ARROW_PYTHON_CONFIG_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/do_import_numpy.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/do_import_numpy.h b/cpp/src/arrow/python/do_import_numpy.h
new file mode 100644
index 0000000..bb4a382
--- /dev/null
+++ b/cpp/src/arrow/python/do_import_numpy.h
@@ -0,0 +1,21 @@
+// 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.
+
+// Trick borrowed from dynd-python for initializing the NumPy array API
+
+// Trigger the array import (inversion of NO_IMPORT_ARRAY)
+#define NUMPY_IMPORT_ARRAY

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/helpers.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/helpers.cc b/cpp/src/arrow/python/helpers.cc
new file mode 100644
index 0000000..add2d9a
--- /dev/null
+++ b/cpp/src/arrow/python/helpers.cc
@@ -0,0 +1,55 @@
+// 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 "arrow/python/helpers.h"
+
+#include <arrow/api.h>
+
+namespace arrow {
+namespace py {
+
+#define GET_PRIMITIVE_TYPE(NAME, FACTORY) \
+  case Type::NAME:                        \
+    return FACTORY();                     \
+    break;
+
+std::shared_ptr<DataType> GetPrimitiveType(Type::type type) {
+  switch (type) {
+    case Type::NA:
+      return null();
+      GET_PRIMITIVE_TYPE(UINT8, uint8);
+      GET_PRIMITIVE_TYPE(INT8, int8);
+      GET_PRIMITIVE_TYPE(UINT16, uint16);
+      GET_PRIMITIVE_TYPE(INT16, int16);
+      GET_PRIMITIVE_TYPE(UINT32, uint32);
+      GET_PRIMITIVE_TYPE(INT32, int32);
+      GET_PRIMITIVE_TYPE(UINT64, uint64);
+      GET_PRIMITIVE_TYPE(INT64, int64);
+      GET_PRIMITIVE_TYPE(DATE32, date32);
+      GET_PRIMITIVE_TYPE(DATE64, date64);
+      GET_PRIMITIVE_TYPE(BOOL, boolean);
+      GET_PRIMITIVE_TYPE(FLOAT, float32);
+      GET_PRIMITIVE_TYPE(DOUBLE, float64);
+      GET_PRIMITIVE_TYPE(BINARY, binary);
+      GET_PRIMITIVE_TYPE(STRING, utf8);
+    default:
+      return nullptr;
+  }
+}
+
+}  // namespace py
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/helpers.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/helpers.h b/cpp/src/arrow/python/helpers.h
new file mode 100644
index 0000000..611e814
--- /dev/null
+++ b/cpp/src/arrow/python/helpers.h
@@ -0,0 +1,35 @@
+// 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.
+
+#ifndef PYARROW_HELPERS_H
+#define PYARROW_HELPERS_H
+
+#include <memory>
+
+#include "arrow/type.h"
+#include "arrow/util/visibility.h"
+
+namespace arrow {
+namespace py {
+
+ARROW_EXPORT
+std::shared_ptr<DataType> GetPrimitiveType(Type::type type);
+
+}  // namespace py
+}  // namespace arrow
+
+#endif  // PYARROW_HELPERS_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/io.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/io.cc b/cpp/src/arrow/python/io.cc
new file mode 100644
index 0000000..ba82a45
--- /dev/null
+++ b/cpp/src/arrow/python/io.cc
@@ -0,0 +1,222 @@
+// 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 "arrow/python/io.h"
+
+#include <cstdint>
+#include <cstdlib>
+#include <string>
+
+#include "arrow/io/memory.h"
+#include "arrow/memory_pool.h"
+#include "arrow/status.h"
+
+#include "arrow/python/common.h"
+
+namespace arrow {
+namespace py {
+
+// ----------------------------------------------------------------------
+// Python file
+
+PythonFile::PythonFile(PyObject* file) : file_(file) {
+  Py_INCREF(file_);
+}
+
+PythonFile::~PythonFile() {
+  Py_DECREF(file_);
+}
+
+static Status CheckPyError() {
+  if (PyErr_Occurred()) {
+    PyObject *exc_type, *exc_value, *traceback;
+    PyErr_Fetch(&exc_type, &exc_value, &traceback);
+    PyObjectStringify stringified(exc_value);
+    std::string message(stringified.bytes);
+    Py_XDECREF(exc_type);
+    Py_XDECREF(exc_value);
+    Py_XDECREF(traceback);
+    PyErr_Clear();
+    return Status::IOError(message);
+  }
+  return Status::OK();
+}
+
+// This is annoying: because C++11 does not allow implicit conversion of string
+// literals to non-const char*, we need to go through some gymnastics to use
+// PyObject_CallMethod without a lot of pain (its arguments are non-const
+// char*)
+template <typename... ArgTypes>
+static inline PyObject* cpp_PyObject_CallMethod(
+    PyObject* obj, const char* method_name, const char* argspec, ArgTypes... args) {
+  return PyObject_CallMethod(
+      obj, const_cast<char*>(method_name), const_cast<char*>(argspec), args...);
+}
+
+Status PythonFile::Close() {
+  // whence: 0 for relative to start of file, 2 for end of file
+  PyObject* result = cpp_PyObject_CallMethod(file_, "close", "()");
+  Py_XDECREF(result);
+  ARROW_RETURN_NOT_OK(CheckPyError());
+  return Status::OK();
+}
+
+Status PythonFile::Seek(int64_t position, int whence) {
+  // whence: 0 for relative to start of file, 2 for end of file
+  PyObject* result = cpp_PyObject_CallMethod(file_, "seek", "(ii)", position, whence);
+  Py_XDECREF(result);
+  ARROW_RETURN_NOT_OK(CheckPyError());
+  return Status::OK();
+}
+
+Status PythonFile::Read(int64_t nbytes, PyObject** out) {
+  PyObject* result = cpp_PyObject_CallMethod(file_, "read", "(i)", nbytes);
+  ARROW_RETURN_NOT_OK(CheckPyError());
+  *out = result;
+  return Status::OK();
+}
+
+Status PythonFile::Write(const uint8_t* data, int64_t nbytes) {
+  PyObject* py_data =
+      PyBytes_FromStringAndSize(reinterpret_cast<const char*>(data), nbytes);
+  ARROW_RETURN_NOT_OK(CheckPyError());
+
+  PyObject* result = cpp_PyObject_CallMethod(file_, "write", "(O)", py_data);
+  Py_XDECREF(py_data);
+  Py_XDECREF(result);
+  ARROW_RETURN_NOT_OK(CheckPyError());
+  return Status::OK();
+}
+
+Status PythonFile::Tell(int64_t* position) {
+  PyObject* result = cpp_PyObject_CallMethod(file_, "tell", "()");
+  ARROW_RETURN_NOT_OK(CheckPyError());
+
+  *position = PyLong_AsLongLong(result);
+  Py_DECREF(result);
+
+  // PyLong_AsLongLong can raise OverflowError
+  ARROW_RETURN_NOT_OK(CheckPyError());
+
+  return Status::OK();
+}
+
+// ----------------------------------------------------------------------
+// Seekable input stream
+
+PyReadableFile::PyReadableFile(PyObject* file) {
+  file_.reset(new PythonFile(file));
+}
+
+PyReadableFile::~PyReadableFile() {}
+
+Status PyReadableFile::Close() {
+  PyAcquireGIL lock;
+  return file_->Close();
+}
+
+Status PyReadableFile::Seek(int64_t position) {
+  PyAcquireGIL lock;
+  return file_->Seek(position, 0);
+}
+
+Status PyReadableFile::Tell(int64_t* position) {
+  PyAcquireGIL lock;
+  return file_->Tell(position);
+}
+
+Status PyReadableFile::Read(int64_t nbytes, int64_t* bytes_read, uint8_t* out) {
+  PyAcquireGIL lock;
+  PyObject* bytes_obj;
+  ARROW_RETURN_NOT_OK(file_->Read(nbytes, &bytes_obj));
+
+  *bytes_read = PyBytes_GET_SIZE(bytes_obj);
+  std::memcpy(out, PyBytes_AS_STRING(bytes_obj), *bytes_read);
+  Py_DECREF(bytes_obj);
+
+  return Status::OK();
+}
+
+Status PyReadableFile::Read(int64_t nbytes, std::shared_ptr<Buffer>* out) {
+  PyAcquireGIL lock;
+
+  PyObject* bytes_obj;
+  ARROW_RETURN_NOT_OK(file_->Read(nbytes, &bytes_obj));
+
+  *out = std::make_shared<PyBuffer>(bytes_obj);
+  Py_DECREF(bytes_obj);
+
+  return Status::OK();
+}
+
+Status PyReadableFile::GetSize(int64_t* size) {
+  PyAcquireGIL lock;
+
+  int64_t current_position;
+
+  ARROW_RETURN_NOT_OK(file_->Tell(&current_position));
+
+  ARROW_RETURN_NOT_OK(file_->Seek(0, 2));
+
+  int64_t file_size;
+  ARROW_RETURN_NOT_OK(file_->Tell(&file_size));
+
+  // Restore previous file position
+  ARROW_RETURN_NOT_OK(file_->Seek(current_position, 0));
+
+  *size = file_size;
+  return Status::OK();
+}
+
+bool PyReadableFile::supports_zero_copy() const {
+  return false;
+}
+
+// ----------------------------------------------------------------------
+// Output stream
+
+PyOutputStream::PyOutputStream(PyObject* file) {
+  file_.reset(new PythonFile(file));
+}
+
+PyOutputStream::~PyOutputStream() {}
+
+Status PyOutputStream::Close() {
+  PyAcquireGIL lock;
+  return file_->Close();
+}
+
+Status PyOutputStream::Tell(int64_t* position) {
+  PyAcquireGIL lock;
+  return file_->Tell(position);
+}
+
+Status PyOutputStream::Write(const uint8_t* data, int64_t nbytes) {
+  PyAcquireGIL lock;
+  return file_->Write(data, nbytes);
+}
+
+// ----------------------------------------------------------------------
+// A readable file that is backed by a PyBuffer
+
+PyBytesReader::PyBytesReader(PyObject* obj)
+    : io::BufferReader(std::make_shared<PyBuffer>(obj)) {}
+
+PyBytesReader::~PyBytesReader() {}
+
+}  // namespace py
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/io.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/io.h b/cpp/src/arrow/python/io.h
new file mode 100644
index 0000000..905bd6c
--- /dev/null
+++ b/cpp/src/arrow/python/io.h
@@ -0,0 +1,99 @@
+// 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.
+
+#ifndef PYARROW_IO_H
+#define PYARROW_IO_H
+
+#include "arrow/io/interfaces.h"
+#include "arrow/io/memory.h"
+#include "arrow/util/visibility.h"
+
+#include "arrow/python/config.h"
+
+#include "arrow/python/common.h"
+
+namespace arrow {
+
+class MemoryPool;
+
+namespace py {
+
+// A common interface to a Python file-like object. Must acquire GIL before
+// calling any methods
+class PythonFile {
+ public:
+  explicit PythonFile(PyObject* file);
+  ~PythonFile();
+
+  Status Close();
+  Status Seek(int64_t position, int whence);
+  Status Read(int64_t nbytes, PyObject** out);
+  Status Tell(int64_t* position);
+  Status Write(const uint8_t* data, int64_t nbytes);
+
+ private:
+  PyObject* file_;
+};
+
+class ARROW_EXPORT PyReadableFile : public io::RandomAccessFile {
+ public:
+  explicit PyReadableFile(PyObject* file);
+  virtual ~PyReadableFile();
+
+  Status Close() override;
+
+  Status Read(int64_t nbytes, int64_t* bytes_read, uint8_t* out) override;
+  Status Read(int64_t nbytes, std::shared_ptr<Buffer>* out) override;
+
+  Status GetSize(int64_t* size) override;
+
+  Status Seek(int64_t position) override;
+
+  Status Tell(int64_t* position) override;
+
+  bool supports_zero_copy() const override;
+
+ private:
+  std::unique_ptr<PythonFile> file_;
+};
+
+class ARROW_EXPORT PyOutputStream : public io::OutputStream {
+ public:
+  explicit PyOutputStream(PyObject* file);
+  virtual ~PyOutputStream();
+
+  Status Close() override;
+  Status Tell(int64_t* position) override;
+  Status Write(const uint8_t* data, int64_t nbytes) override;
+
+ private:
+  std::unique_ptr<PythonFile> file_;
+};
+
+// A zero-copy reader backed by a PyBuffer object
+class ARROW_EXPORT PyBytesReader : public io::BufferReader {
+ public:
+  explicit PyBytesReader(PyObject* obj);
+  virtual ~PyBytesReader();
+};
+
+// TODO(wesm): seekable output files
+
+}  // namespace py
+}  // namespace arrow
+
+#endif  // PYARROW_IO_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/cpp/src/arrow/python/numpy_interop.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/numpy_interop.h b/cpp/src/arrow/python/numpy_interop.h
new file mode 100644
index 0000000..0a4b425
--- /dev/null
+++ b/cpp/src/arrow/python/numpy_interop.h
@@ -0,0 +1,60 @@
+// 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.
+
+#ifndef PYARROW_NUMPY_INTEROP_H
+#define PYARROW_NUMPY_INTEROP_H
+
+#include <Python.h>
+
+#include <numpy/numpyconfig.h>
+
+// Don't use the deprecated Numpy functions
+#ifdef NPY_1_7_API_VERSION
+#define NPY_NO_DEPRECATED_API NPY_1_7_API_VERSION
+#else
+#define NPY_ARRAY_NOTSWAPPED NPY_NOTSWAPPED
+#define NPY_ARRAY_ALIGNED NPY_ALIGNED
+#define NPY_ARRAY_WRITEABLE NPY_WRITEABLE
+#define NPY_ARRAY_UPDATEIFCOPY NPY_UPDATEIFCOPY
+#endif
+
+// This is required to be able to access the NumPy C API properly in C++ files
+// other than this main one
+#define PY_ARRAY_UNIQUE_SYMBOL arrow_ARRAY_API
+#ifndef NUMPY_IMPORT_ARRAY
+#define NO_IMPORT_ARRAY
+#endif
+
+#include <numpy/arrayobject.h>
+#include <numpy/ufuncobject.h>
+
+namespace arrow {
+namespace py {
+
+inline int import_numpy() {
+#ifdef NUMPY_IMPORT_ARRAY
+  import_array1(-1);
+  import_umath1(-1);
+#endif
+
+  return 0;
+}
+
+}  // namespace py
+}  // namespace arrow
+
+#endif  // PYARROW_NUMPY_INTEROP_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/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
new file mode 100644
index 0000000..ae2527e
--- /dev/null
+++ b/cpp/src/arrow/python/pandas-test.cc
@@ -0,0 +1,64 @@
+// 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/python/pandas_convert.h"
+#include "arrow/schema.h"
+#include "arrow/table.h"
+#include "arrow/test-util.h"
+#include "arrow/type.h"
+
+namespace arrow {
+namespace py {
+
+TEST(PandasConversionTest, TestObjectBlockWriteFails) {
+  StringBuilder builder(default_memory_pool());
+  const char value[] = {'\xf1', '\0'};
+
+  for (int i = 0; i < 1000; ++i) {
+    builder.Append(value, 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


[2/5] arrow git commit: ARROW-341: [Python] Move pyarrow's C++ code to the main C++ source tree, install libarrow_python and headers

Posted by we...@apache.org.
http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/adapters/pandas.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/pandas.cc b/python/src/pyarrow/adapters/pandas.cc
deleted file mode 100644
index a7386ce..0000000
--- a/python/src/pyarrow/adapters/pandas.cc
+++ /dev/null
@@ -1,1936 +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.
-
-// Functions for pandas conversion via NumPy
-
-#include <Python.h>
-
-#include "pyarrow/adapters/pandas.h"
-#include "pyarrow/numpy_interop.h"
-
-#include <algorithm>
-#include <atomic>
-#include <cmath>
-#include <cstdint>
-#include <memory>
-#include <mutex>
-#include <sstream>
-#include <string>
-#include <thread>
-#include <unordered_map>
-
-#include "arrow/array.h"
-#include "arrow/column.h"
-#include "arrow/loader.h"
-#include "arrow/status.h"
-#include "arrow/table.h"
-#include "arrow/type_fwd.h"
-#include "arrow/type_traits.h"
-#include "arrow/util/bit-util.h"
-#include "arrow/util/macros.h"
-
-#include "pyarrow/adapters/builtin.h"
-#include "pyarrow/common.h"
-#include "pyarrow/config.h"
-#include "pyarrow/type_traits.h"
-#include "pyarrow/util/datetime.h"
-
-namespace arrow {
-namespace py {
-
-// ----------------------------------------------------------------------
-// Utility code
-
-int cast_npy_type_compat(int type_num) {
-// Both LONGLONG and INT64 can be observed in the wild, which is buggy. We set
-// U/LONGLONG to U/INT64 so things work properly.
-
-#if (NPY_INT64 == NPY_LONGLONG) && (NPY_SIZEOF_LONGLONG == 8)
-  if (type_num == NPY_LONGLONG) { type_num = NPY_INT64; }
-  if (type_num == NPY_ULONGLONG) { type_num = NPY_UINT64; }
-#endif
-
-  return type_num;
-}
-
-static inline bool PyObject_is_null(const PyObject* obj) {
-  return obj == Py_None || obj == numpy_nan;
-}
-
-static inline bool PyObject_is_string(const PyObject* obj) {
-#if PY_MAJOR_VERSION >= 3
-  return PyUnicode_Check(obj) || PyBytes_Check(obj);
-#else
-  return PyString_Check(obj) || PyUnicode_Check(obj);
-#endif
-}
-
-template <int TYPE>
-static int64_t ValuesToBitmap(const void* data, int64_t length, uint8_t* bitmap) {
-  typedef npy_traits<TYPE> traits;
-  typedef typename traits::value_type T;
-
-  int64_t null_count = 0;
-  const T* values = reinterpret_cast<const T*>(data);
-
-  // TODO(wesm): striding
-  for (int i = 0; i < length; ++i) {
-    if (traits::isnull(values[i])) {
-      ++null_count;
-    } else {
-      BitUtil::SetBit(bitmap, i);
-    }
-  }
-
-  return null_count;
-}
-
-// Returns null count
-static int64_t MaskToBitmap(PyArrayObject* mask, int64_t length, uint8_t* bitmap) {
-  int64_t null_count = 0;
-  const uint8_t* mask_values = static_cast<const uint8_t*>(PyArray_DATA(mask));
-  // TODO(wesm): strided null mask
-  for (int i = 0; i < length; ++i) {
-    if (mask_values[i]) {
-      ++null_count;
-    } else {
-      BitUtil::SetBit(bitmap, i);
-    }
-  }
-  return null_count;
-}
-
-template <int TYPE>
-static int64_t ValuesToValidBytes(
-    const void* data, int64_t length, uint8_t* valid_bytes) {
-  typedef npy_traits<TYPE> traits;
-  typedef typename traits::value_type T;
-
-  int64_t null_count = 0;
-  const T* values = reinterpret_cast<const T*>(data);
-
-  // TODO(wesm): striding
-  for (int i = 0; i < length; ++i) {
-    valid_bytes[i] = not traits::isnull(values[i]);
-    if (traits::isnull(values[i])) null_count++;
-  }
-
-  return null_count;
-}
-
-Status CheckFlatNumpyArray(PyArrayObject* numpy_array, int np_type) {
-  if (PyArray_NDIM(numpy_array) != 1) {
-    return Status::Invalid("only handle 1-dimensional arrays");
-  }
-
-  if (PyArray_DESCR(numpy_array)->type_num != np_type) {
-    return Status::Invalid("can only handle exact conversions");
-  }
-
-  npy_intp* astrides = PyArray_STRIDES(numpy_array);
-  if (astrides[0] != PyArray_DESCR(numpy_array)->elsize) {
-    return Status::Invalid("No support for strided arrays in lists yet");
-  }
-  return Status::OK();
-}
-
-Status AppendObjectStrings(StringBuilder& string_builder, PyObject** objects,
-    int64_t objects_length, bool* have_bytes) {
-  PyObject* obj;
-
-  for (int64_t i = 0; i < objects_length; ++i) {
-    obj = objects[i];
-    if (PyUnicode_Check(obj)) {
-      obj = PyUnicode_AsUTF8String(obj);
-      if (obj == NULL) {
-        PyErr_Clear();
-        return Status::TypeError("failed converting unicode to UTF8");
-      }
-      const int64_t length = PyBytes_GET_SIZE(obj);
-      Status s = string_builder.Append(PyBytes_AS_STRING(obj), length);
-      Py_DECREF(obj);
-      if (!s.ok()) { return s; }
-    } else if (PyBytes_Check(obj)) {
-      *have_bytes = true;
-      const int64_t length = PyBytes_GET_SIZE(obj);
-      RETURN_NOT_OK(string_builder.Append(PyBytes_AS_STRING(obj), length));
-    } else {
-      string_builder.AppendNull();
-    }
-  }
-
-  return Status::OK();
-}
-
-template <typename T>
-struct WrapBytes {};
-
-template <>
-struct WrapBytes<StringArray> {
-  static inline PyObject* Wrap(const uint8_t* data, int64_t length) {
-    return PyUnicode_FromStringAndSize(reinterpret_cast<const char*>(data), length);
-  }
-};
-
-template <>
-struct WrapBytes<BinaryArray> {
-  static inline PyObject* Wrap(const uint8_t* data, int64_t length) {
-    return PyBytes_FromStringAndSize(reinterpret_cast<const char*>(data), length);
-  }
-};
-
-static inline bool ListTypeSupported(const Type::type type_id) {
-  switch (type_id) {
-    case Type::UINT8:
-    case Type::INT8:
-    case Type::UINT16:
-    case Type::INT16:
-    case Type::UINT32:
-    case Type::INT32:
-    case Type::INT64:
-    case Type::UINT64:
-    case Type::FLOAT:
-    case Type::DOUBLE:
-    case Type::STRING:
-    case Type::TIMESTAMP:
-      // The above types are all supported.
-      return true;
-    default:
-      break;
-  }
-  return false;
-}
-
-// ----------------------------------------------------------------------
-// Conversion from NumPy-in-Pandas to Arrow
-
-class PandasConverter : public TypeVisitor {
- public:
-  PandasConverter(
-      MemoryPool* pool, PyObject* ao, PyObject* mo, const std::shared_ptr<DataType>& type)
-      : pool_(pool),
-        type_(type),
-        arr_(reinterpret_cast<PyArrayObject*>(ao)),
-        mask_(nullptr) {
-    if (mo != nullptr and mo != Py_None) { mask_ = reinterpret_cast<PyArrayObject*>(mo); }
-    length_ = PyArray_SIZE(arr_);
-  }
-
-  bool is_strided() const {
-    npy_intp* astrides = PyArray_STRIDES(arr_);
-    return astrides[0] != PyArray_DESCR(arr_)->elsize;
-  }
-
-  Status InitNullBitmap() {
-    int null_bytes = BitUtil::BytesForBits(length_);
-
-    null_bitmap_ = std::make_shared<PoolBuffer>(pool_);
-    RETURN_NOT_OK(null_bitmap_->Resize(null_bytes));
-
-    null_bitmap_data_ = null_bitmap_->mutable_data();
-    memset(null_bitmap_data_, 0, null_bytes);
-
-    return Status::OK();
-  }
-
-  // ----------------------------------------------------------------------
-  // Traditional visitor conversion for non-object arrays
-
-  template <typename ArrowType>
-  Status ConvertData(std::shared_ptr<Buffer>* data);
-
-  template <typename ArrowType>
-  Status VisitNative() {
-    using traits = arrow_traits<ArrowType::type_id>;
-
-    if (mask_ != nullptr || traits::supports_nulls) { RETURN_NOT_OK(InitNullBitmap()); }
-
-    std::shared_ptr<Buffer> data;
-    RETURN_NOT_OK(ConvertData<ArrowType>(&data));
-
-    int64_t null_count = 0;
-    if (mask_ != nullptr) {
-      null_count = MaskToBitmap(mask_, length_, null_bitmap_data_);
-    } else if (traits::supports_nulls) {
-      // TODO(wesm): this presumes the NumPy C type and arrow C type are the
-      // same
-      null_count = ValuesToBitmap<traits::npy_type>(
-          PyArray_DATA(arr_), length_, null_bitmap_data_);
-    }
-
-    std::vector<FieldMetadata> fields(1);
-    fields[0].length = length_;
-    fields[0].null_count = null_count;
-    fields[0].offset = 0;
-
-    return LoadArray(type_, fields, {null_bitmap_, data}, &out_);
-  }
-
-#define VISIT_NATIVE(TYPE) \
-  Status Visit(const TYPE& type) override { return VisitNative<TYPE>(); }
-
-  VISIT_NATIVE(BooleanType);
-  VISIT_NATIVE(Int8Type);
-  VISIT_NATIVE(Int16Type);
-  VISIT_NATIVE(Int32Type);
-  VISIT_NATIVE(Int64Type);
-  VISIT_NATIVE(UInt8Type);
-  VISIT_NATIVE(UInt16Type);
-  VISIT_NATIVE(UInt32Type);
-  VISIT_NATIVE(UInt64Type);
-  VISIT_NATIVE(FloatType);
-  VISIT_NATIVE(DoubleType);
-  VISIT_NATIVE(TimestampType);
-
-#undef VISIT_NATIVE
-
-  Status Convert(std::shared_ptr<Array>* out) {
-    if (PyArray_NDIM(arr_) != 1) {
-      return Status::Invalid("only handle 1-dimensional arrays");
-    }
-    // TODO(wesm): strided arrays
-    if (is_strided()) { return Status::Invalid("no support for strided data yet"); }
-
-    if (type_ == nullptr) { return Status::Invalid("Must pass data type"); }
-
-    // Visit the type to perform conversion
-    RETURN_NOT_OK(type_->Accept(this));
-
-    *out = out_;
-    return Status::OK();
-  }
-
-  // ----------------------------------------------------------------------
-  // Conversion logic for various object dtype arrays
-
-  template <int ITEM_TYPE, typename ArrowType>
-  Status ConvertTypedLists(
-      const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out);
-
-  Status ConvertObjectStrings(std::shared_ptr<Array>* out);
-  Status ConvertBooleans(std::shared_ptr<Array>* out);
-  Status ConvertDates(std::shared_ptr<Array>* out);
-  Status ConvertLists(const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out);
-  Status ConvertObjects(std::shared_ptr<Array>* out);
-
- protected:
-  MemoryPool* pool_;
-  std::shared_ptr<DataType> type_;
-  PyArrayObject* arr_;
-  PyArrayObject* mask_;
-  int64_t length_;
-
-  // Used in visitor pattern
-  std::shared_ptr<Array> out_;
-
-  std::shared_ptr<ResizableBuffer> null_bitmap_;
-  uint8_t* null_bitmap_data_;
-};
-
-template <typename ArrowType>
-inline Status PandasConverter::ConvertData(std::shared_ptr<Buffer>* data) {
-  using traits = arrow_traits<ArrowType::type_id>;
-
-  // Handle LONGLONG->INT64 and other fun things
-  int type_num_compat = cast_npy_type_compat(PyArray_DESCR(arr_)->type_num);
-
-  if (traits::npy_type != type_num_compat) {
-    return Status::NotImplemented("NumPy type casts not yet implemented");
-  }
-
-  *data = std::make_shared<NumPyBuffer>(arr_);
-  return Status::OK();
-}
-
-template <>
-inline Status PandasConverter::ConvertData<BooleanType>(std::shared_ptr<Buffer>* data) {
-  int nbytes = BitUtil::BytesForBits(length_);
-  auto buffer = std::make_shared<PoolBuffer>(pool_);
-  RETURN_NOT_OK(buffer->Resize(nbytes));
-
-  const uint8_t* values = reinterpret_cast<const uint8_t*>(PyArray_DATA(arr_));
-
-  uint8_t* bitmap = buffer->mutable_data();
-
-  memset(bitmap, 0, nbytes);
-  for (int i = 0; i < length_; ++i) {
-    if (values[i] > 0) { BitUtil::SetBit(bitmap, i); }
-  }
-
-  *data = buffer;
-  return Status::OK();
-}
-
-Status PandasConverter::ConvertDates(std::shared_ptr<Array>* out) {
-  PyAcquireGIL lock;
-
-  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-  Date64Builder date_builder(pool_);
-  RETURN_NOT_OK(date_builder.Resize(length_));
-
-  Status s;
-  PyObject* obj;
-  for (int64_t i = 0; i < length_; ++i) {
-    obj = objects[i];
-    if (PyDate_CheckExact(obj)) {
-      PyDateTime_Date* pydate = reinterpret_cast<PyDateTime_Date*>(obj);
-      date_builder.Append(PyDate_to_ms(pydate));
-    } else {
-      date_builder.AppendNull();
-    }
-  }
-  return date_builder.Finish(out);
-}
-
-Status PandasConverter::ConvertObjectStrings(std::shared_ptr<Array>* out) {
-  PyAcquireGIL lock;
-
-  // The output type at this point is inconclusive because there may be bytes
-  // and unicode mixed in the object array
-
-  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-  StringBuilder string_builder(pool_);
-  RETURN_NOT_OK(string_builder.Resize(length_));
-
-  Status s;
-  bool have_bytes = false;
-  RETURN_NOT_OK(AppendObjectStrings(string_builder, objects, length_, &have_bytes));
-  RETURN_NOT_OK(string_builder.Finish(out));
-
-  if (have_bytes) {
-    const auto& arr = static_cast<const StringArray&>(*out->get());
-    *out = std::make_shared<BinaryArray>(arr.length(), arr.value_offsets(), arr.data(),
-        arr.null_bitmap(), arr.null_count());
-  }
-  return Status::OK();
-}
-
-Status PandasConverter::ConvertBooleans(std::shared_ptr<Array>* out) {
-  PyAcquireGIL lock;
-
-  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-
-  int nbytes = BitUtil::BytesForBits(length_);
-  auto data = std::make_shared<PoolBuffer>(pool_);
-  RETURN_NOT_OK(data->Resize(nbytes));
-  uint8_t* bitmap = data->mutable_data();
-  memset(bitmap, 0, nbytes);
-
-  int64_t null_count = 0;
-  for (int64_t i = 0; i < length_; ++i) {
-    if (objects[i] == Py_True) {
-      BitUtil::SetBit(bitmap, i);
-      BitUtil::SetBit(null_bitmap_data_, i);
-    } else if (objects[i] != Py_False) {
-      ++null_count;
-    } else {
-      BitUtil::SetBit(null_bitmap_data_, i);
-    }
-  }
-
-  *out = std::make_shared<BooleanArray>(length_, data, null_bitmap_, null_count);
-
-  return Status::OK();
-}
-
-Status PandasConverter::ConvertObjects(std::shared_ptr<Array>* out) {
-  // Python object arrays are annoying, since we could have one of:
-  //
-  // * Strings
-  // * Booleans with nulls
-  // * Mixed type (not supported at the moment by arrow format)
-  //
-  // Additionally, nulls may be encoded either as np.nan or None. So we have to
-  // do some type inference and conversion
-
-  RETURN_NOT_OK(InitNullBitmap());
-
-  // TODO: mask not supported here
-  if (mask_ != nullptr) {
-    return Status::NotImplemented("mask not supported in object conversions yet");
-  }
-
-  const PyObject** objects;
-  {
-    PyAcquireGIL lock;
-    objects = reinterpret_cast<const PyObject**>(PyArray_DATA(arr_));
-    PyDateTime_IMPORT;
-  }
-
-  if (type_) {
-    switch (type_->type) {
-      case Type::STRING:
-        return ConvertObjectStrings(out);
-      case Type::BOOL:
-        return ConvertBooleans(out);
-      case Type::DATE64:
-        return ConvertDates(out);
-      case Type::LIST: {
-        const auto& list_field = static_cast<const ListType&>(*type_);
-        return ConvertLists(list_field.value_field()->type, out);
-      }
-      default:
-        return Status::TypeError("No known conversion to Arrow type");
-    }
-  } else {
-    for (int64_t i = 0; i < length_; ++i) {
-      if (PyObject_is_null(objects[i])) {
-        continue;
-      } else if (PyObject_is_string(objects[i])) {
-        return ConvertObjectStrings(out);
-      } else if (PyBool_Check(objects[i])) {
-        return ConvertBooleans(out);
-      } else if (PyDate_CheckExact(objects[i])) {
-        return ConvertDates(out);
-      } else {
-        return Status::TypeError("unhandled python type");
-      }
-    }
-  }
-
-  return Status::TypeError("Unable to infer type of object array, were all null");
-}
-
-template <int ITEM_TYPE, typename ArrowType>
-inline Status PandasConverter::ConvertTypedLists(
-    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
-  typedef npy_traits<ITEM_TYPE> traits;
-  typedef typename traits::value_type T;
-  typedef typename traits::BuilderClass BuilderT;
-
-  PyAcquireGIL lock;
-
-  auto value_builder = std::make_shared<BuilderT>(pool_, type);
-  ListBuilder list_builder(pool_, value_builder);
-  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-  for (int64_t i = 0; i < length_; ++i) {
-    if (PyObject_is_null(objects[i])) {
-      RETURN_NOT_OK(list_builder.AppendNull());
-    } else if (PyArray_Check(objects[i])) {
-      auto numpy_array = reinterpret_cast<PyArrayObject*>(objects[i]);
-      RETURN_NOT_OK(list_builder.Append(true));
-
-      // TODO(uwe): Support more complex numpy array structures
-      RETURN_NOT_OK(CheckFlatNumpyArray(numpy_array, ITEM_TYPE));
-
-      int64_t size = PyArray_DIM(numpy_array, 0);
-      auto data = reinterpret_cast<const T*>(PyArray_DATA(numpy_array));
-      if (traits::supports_nulls) {
-        null_bitmap_->Resize(size, false);
-        // TODO(uwe): A bitmap would be more space-efficient but the Builder API doesn't
-        // currently support this.
-        // ValuesToBitmap<ITEM_TYPE>(data, size, null_bitmap_->mutable_data());
-        ValuesToValidBytes<ITEM_TYPE>(data, size, null_bitmap_->mutable_data());
-        RETURN_NOT_OK(value_builder->Append(data, size, null_bitmap_->data()));
-      } else {
-        RETURN_NOT_OK(value_builder->Append(data, size));
-      }
-
-    } else if (PyList_Check(objects[i])) {
-      int64_t size;
-      std::shared_ptr<DataType> inferred_type;
-      RETURN_NOT_OK(list_builder.Append(true));
-      RETURN_NOT_OK(InferArrowType(objects[i], &size, &inferred_type));
-      if (inferred_type->type != type->type) {
-        std::stringstream ss;
-        ss << inferred_type->ToString() << " cannot be converted to " << type->ToString();
-        return Status::TypeError(ss.str());
-      }
-      RETURN_NOT_OK(AppendPySequence(objects[i], type, value_builder));
-    } else {
-      return Status::TypeError("Unsupported Python type for list items");
-    }
-  }
-  return list_builder.Finish(out);
-}
-
-template <>
-inline Status PandasConverter::ConvertTypedLists<NPY_OBJECT, StringType>(
-    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
-  PyAcquireGIL lock;
-  // TODO: If there are bytes involed, convert to Binary representation
-  bool have_bytes = false;
-
-  auto value_builder = std::make_shared<StringBuilder>(pool_);
-  ListBuilder list_builder(pool_, value_builder);
-  PyObject** objects = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-  for (int64_t i = 0; i < length_; ++i) {
-    if (PyObject_is_null(objects[i])) {
-      RETURN_NOT_OK(list_builder.AppendNull());
-    } else if (PyArray_Check(objects[i])) {
-      auto numpy_array = reinterpret_cast<PyArrayObject*>(objects[i]);
-      RETURN_NOT_OK(list_builder.Append(true));
-
-      // TODO(uwe): Support more complex numpy array structures
-      RETURN_NOT_OK(CheckFlatNumpyArray(numpy_array, NPY_OBJECT));
-
-      int64_t size = PyArray_DIM(numpy_array, 0);
-      auto data = reinterpret_cast<PyObject**>(PyArray_DATA(numpy_array));
-      RETURN_NOT_OK(AppendObjectStrings(*value_builder.get(), data, size, &have_bytes));
-    } else if (PyList_Check(objects[i])) {
-      int64_t size;
-      std::shared_ptr<DataType> inferred_type;
-      RETURN_NOT_OK(list_builder.Append(true));
-      RETURN_NOT_OK(InferArrowType(objects[i], &size, &inferred_type));
-      if (inferred_type->type != Type::STRING) {
-        std::stringstream ss;
-        ss << inferred_type->ToString() << " cannot be converted to STRING.";
-        return Status::TypeError(ss.str());
-      }
-      RETURN_NOT_OK(AppendPySequence(objects[i], inferred_type, value_builder));
-    } else {
-      return Status::TypeError("Unsupported Python type for list items");
-    }
-  }
-  return list_builder.Finish(out);
-}
-
-#define LIST_CASE(TYPE, NUMPY_TYPE, ArrowType)                  \
-  case Type::TYPE: {                                            \
-    return ConvertTypedLists<NUMPY_TYPE, ArrowType>(type, out); \
-  }
-
-Status PandasConverter::ConvertLists(
-    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
-  switch (type->type) {
-    LIST_CASE(UINT8, NPY_UINT8, UInt8Type)
-    LIST_CASE(INT8, NPY_INT8, Int8Type)
-    LIST_CASE(UINT16, NPY_UINT16, UInt16Type)
-    LIST_CASE(INT16, NPY_INT16, Int16Type)
-    LIST_CASE(UINT32, NPY_UINT32, UInt32Type)
-    LIST_CASE(INT32, NPY_INT32, Int32Type)
-    LIST_CASE(UINT64, NPY_UINT64, UInt64Type)
-    LIST_CASE(INT64, NPY_INT64, Int64Type)
-    LIST_CASE(TIMESTAMP, NPY_DATETIME, TimestampType)
-    LIST_CASE(FLOAT, NPY_FLOAT, FloatType)
-    LIST_CASE(DOUBLE, NPY_DOUBLE, DoubleType)
-    LIST_CASE(STRING, NPY_OBJECT, StringType)
-    default:
-      return Status::TypeError("Unknown list item type");
-  }
-
-  return Status::TypeError("Unknown list type");
-}
-
-Status PandasToArrow(MemoryPool* pool, PyObject* ao, PyObject* mo,
-    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
-  PandasConverter converter(pool, ao, mo, type);
-  return converter.Convert(out);
-}
-
-Status PandasObjectsToArrow(MemoryPool* pool, PyObject* ao, PyObject* mo,
-    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out) {
-  PandasConverter converter(pool, ao, mo, type);
-  return converter.ConvertObjects(out);
-}
-
-Status PandasDtypeToArrow(PyObject* dtype, std::shared_ptr<DataType>* out) {
-  PyArray_Descr* descr = reinterpret_cast<PyArray_Descr*>(dtype);
-
-  int type_num = cast_npy_type_compat(descr->type_num);
-
-#define TO_ARROW_TYPE_CASE(NPY_NAME, FACTORY) \
-  case NPY_##NPY_NAME:                        \
-    *out = FACTORY();                         \
-    break;
-
-  switch (type_num) {
-    TO_ARROW_TYPE_CASE(BOOL, boolean);
-    TO_ARROW_TYPE_CASE(INT8, int8);
-    TO_ARROW_TYPE_CASE(INT16, int16);
-    TO_ARROW_TYPE_CASE(INT32, int32);
-    TO_ARROW_TYPE_CASE(INT64, int64);
-#if (NPY_INT64 != NPY_LONGLONG)
-    TO_ARROW_TYPE_CASE(LONGLONG, int64);
-#endif
-    TO_ARROW_TYPE_CASE(UINT8, uint8);
-    TO_ARROW_TYPE_CASE(UINT16, uint16);
-    TO_ARROW_TYPE_CASE(UINT32, uint32);
-    TO_ARROW_TYPE_CASE(UINT64, uint64);
-#if (NPY_UINT64 != NPY_ULONGLONG)
-    TO_ARROW_CASE(ULONGLONG);
-#endif
-    TO_ARROW_TYPE_CASE(FLOAT32, float32);
-    TO_ARROW_TYPE_CASE(FLOAT64, float64);
-    case NPY_DATETIME: {
-      auto date_dtype =
-          reinterpret_cast<PyArray_DatetimeDTypeMetaData*>(descr->c_metadata);
-      TimeUnit unit;
-      switch (date_dtype->meta.base) {
-        case NPY_FR_s:
-          unit = TimeUnit::SECOND;
-          break;
-        case NPY_FR_ms:
-          unit = TimeUnit::MILLI;
-          break;
-        case NPY_FR_us:
-          unit = TimeUnit::MICRO;
-          break;
-        case NPY_FR_ns:
-          unit = TimeUnit::NANO;
-          break;
-        default:
-          return Status::NotImplemented("Unsupported datetime64 time unit");
-      }
-      *out = timestamp(unit);
-    } break;
-    default: {
-      std::stringstream ss;
-      ss << "Unsupported numpy type " << descr->type_num << std::endl;
-      return Status::NotImplemented(ss.str());
-    }
-  }
-
-#undef TO_ARROW_TYPE_CASE
-
-  return Status::OK();
-}
-
-// ----------------------------------------------------------------------
-// pandas 0.x DataFrame conversion internals
-
-inline void set_numpy_metadata(int type, DataType* datatype, PyArrayObject* out) {
-  if (type == NPY_DATETIME) {
-    PyArray_Descr* descr = PyArray_DESCR(out);
-    auto date_dtype = reinterpret_cast<PyArray_DatetimeDTypeMetaData*>(descr->c_metadata);
-    if (datatype->type == Type::TIMESTAMP) {
-      auto timestamp_type = static_cast<TimestampType*>(datatype);
-
-      switch (timestamp_type->unit) {
-        case TimestampType::Unit::SECOND:
-          date_dtype->meta.base = NPY_FR_s;
-          break;
-        case TimestampType::Unit::MILLI:
-          date_dtype->meta.base = NPY_FR_ms;
-          break;
-        case TimestampType::Unit::MICRO:
-          date_dtype->meta.base = NPY_FR_us;
-          break;
-        case TimestampType::Unit::NANO:
-          date_dtype->meta.base = NPY_FR_ns;
-          break;
-      }
-    } else {
-      // datatype->type == Type::DATE64
-      date_dtype->meta.base = NPY_FR_D;
-    }
-  }
-}
-
-class PandasBlock {
- public:
-  enum type {
-    OBJECT,
-    UINT8,
-    INT8,
-    UINT16,
-    INT16,
-    UINT32,
-    INT32,
-    UINT64,
-    INT64,
-    FLOAT,
-    DOUBLE,
-    BOOL,
-    DATETIME,
-    DATETIME_WITH_TZ,
-    CATEGORICAL
-  };
-
-  PandasBlock(int64_t num_rows, int num_columns)
-      : num_rows_(num_rows), num_columns_(num_columns) {}
-  virtual ~PandasBlock() {}
-
-  virtual Status Allocate() = 0;
-  virtual Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
-      int64_t rel_placement) = 0;
-
-  PyObject* block_arr() const { return block_arr_.obj(); }
-
-  virtual Status GetPyResult(PyObject** output) {
-    PyObject* result = PyDict_New();
-    RETURN_IF_PYERROR();
-
-    PyDict_SetItemString(result, "block", block_arr_.obj());
-    PyDict_SetItemString(result, "placement", placement_arr_.obj());
-
-    *output = result;
-
-    return Status::OK();
-  }
-
- protected:
-  Status AllocateNDArray(int npy_type, int ndim = 2) {
-    PyAcquireGIL lock;
-
-    PyObject* block_arr;
-    if (ndim == 2) {
-      npy_intp block_dims[2] = {num_columns_, num_rows_};
-      block_arr = PyArray_SimpleNew(2, block_dims, npy_type);
-    } else {
-      npy_intp block_dims[1] = {num_rows_};
-      block_arr = PyArray_SimpleNew(1, block_dims, npy_type);
-    }
-
-    if (block_arr == NULL) {
-      // TODO(wesm): propagating Python exception
-      return Status::OK();
-    }
-
-    npy_intp placement_dims[1] = {num_columns_};
-    PyObject* placement_arr = PyArray_SimpleNew(1, placement_dims, NPY_INT64);
-    if (placement_arr == NULL) {
-      // TODO(wesm): propagating Python exception
-      return Status::OK();
-    }
-
-    block_arr_.reset(block_arr);
-    placement_arr_.reset(placement_arr);
-
-    block_data_ = reinterpret_cast<uint8_t*>(
-        PyArray_DATA(reinterpret_cast<PyArrayObject*>(block_arr)));
-
-    placement_data_ = reinterpret_cast<int64_t*>(
-        PyArray_DATA(reinterpret_cast<PyArrayObject*>(placement_arr)));
-
-    return Status::OK();
-  }
-
-  int64_t num_rows_;
-  int num_columns_;
-
-  OwnedRef block_arr_;
-  uint8_t* block_data_;
-
-  // ndarray<int32>
-  OwnedRef placement_arr_;
-  int64_t* placement_data_;
-
-  DISALLOW_COPY_AND_ASSIGN(PandasBlock);
-};
-
-template <typename T>
-inline void ConvertIntegerWithNulls(const ChunkedArray& data, double* out_values) {
-  for (int c = 0; c < data.num_chunks(); c++) {
-    const std::shared_ptr<Array> arr = data.chunk(c);
-    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
-    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
-    // Upcast to double, set NaN as appropriate
-
-    for (int i = 0; i < arr->length(); ++i) {
-      *out_values++ = prim_arr->IsNull(i) ? NAN : in_values[i];
-    }
-  }
-}
-
-template <typename T>
-inline void ConvertIntegerNoNullsSameType(const ChunkedArray& data, T* out_values) {
-  for (int c = 0; c < data.num_chunks(); c++) {
-    const std::shared_ptr<Array> arr = data.chunk(c);
-    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
-    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
-    memcpy(out_values, in_values, sizeof(T) * arr->length());
-    out_values += arr->length();
-  }
-}
-
-template <typename InType, typename OutType>
-inline void ConvertIntegerNoNullsCast(const ChunkedArray& data, OutType* out_values) {
-  for (int c = 0; c < data.num_chunks(); c++) {
-    const std::shared_ptr<Array> arr = data.chunk(c);
-    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
-    auto in_values = reinterpret_cast<const InType*>(prim_arr->data()->data());
-    for (int64_t i = 0; i < arr->length(); ++i) {
-      *out_values = in_values[i];
-    }
-  }
-}
-
-static Status ConvertBooleanWithNulls(const ChunkedArray& data, PyObject** out_values) {
-  PyAcquireGIL lock;
-  for (int c = 0; c < data.num_chunks(); c++) {
-    const std::shared_ptr<Array> arr = data.chunk(c);
-    auto bool_arr = static_cast<BooleanArray*>(arr.get());
-
-    for (int64_t i = 0; i < arr->length(); ++i) {
-      if (bool_arr->IsNull(i)) {
-        Py_INCREF(Py_None);
-        *out_values++ = Py_None;
-      } else if (bool_arr->Value(i)) {
-        // True
-        Py_INCREF(Py_True);
-        *out_values++ = Py_True;
-      } else {
-        // False
-        Py_INCREF(Py_False);
-        *out_values++ = Py_False;
-      }
-    }
-  }
-  return Status::OK();
-}
-
-static void ConvertBooleanNoNulls(const ChunkedArray& data, uint8_t* out_values) {
-  for (int c = 0; c < data.num_chunks(); c++) {
-    const std::shared_ptr<Array> arr = data.chunk(c);
-    auto bool_arr = static_cast<BooleanArray*>(arr.get());
-    for (int64_t i = 0; i < arr->length(); ++i) {
-      *out_values++ = static_cast<uint8_t>(bool_arr->Value(i));
-    }
-  }
-}
-
-template <typename ArrayType>
-inline Status ConvertBinaryLike(const ChunkedArray& data, PyObject** out_values) {
-  PyAcquireGIL lock;
-  for (int c = 0; c < data.num_chunks(); c++) {
-    auto arr = static_cast<ArrayType*>(data.chunk(c).get());
-
-    const uint8_t* data_ptr;
-    int32_t length;
-    const bool has_nulls = data.null_count() > 0;
-    for (int64_t i = 0; i < arr->length(); ++i) {
-      if (has_nulls && arr->IsNull(i)) {
-        Py_INCREF(Py_None);
-        *out_values = Py_None;
-      } else {
-        data_ptr = arr->GetValue(i, &length);
-        *out_values = WrapBytes<ArrayType>::Wrap(data_ptr, length);
-        if (*out_values == nullptr) {
-          PyErr_Clear();
-          std::stringstream ss;
-          ss << "Wrapping "
-             << std::string(reinterpret_cast<const char*>(data_ptr), length) << " failed";
-          return Status::UnknownError(ss.str());
-        }
-      }
-      ++out_values;
-    }
-  }
-  return Status::OK();
-}
-
-template <typename ArrowType>
-inline Status ConvertListsLike(
-    const std::shared_ptr<Column>& col, PyObject** out_values) {
-  const ChunkedArray& data = *col->data().get();
-  auto list_type = std::static_pointer_cast<ListType>(col->type());
-
-  // Get column of underlying value arrays
-  std::vector<std::shared_ptr<Array>> value_arrays;
-  for (int c = 0; c < data.num_chunks(); c++) {
-    auto arr = std::static_pointer_cast<ListArray>(data.chunk(c));
-    value_arrays.emplace_back(arr->values());
-  }
-  auto flat_column = std::make_shared<Column>(list_type->value_field(), value_arrays);
-  // TODO(ARROW-489): Currently we don't have a Python reference for single columns.
-  //    Storing a reference to the whole Array would be to expensive.
-  PyObject* numpy_array;
-  RETURN_NOT_OK(ConvertColumnToPandas(flat_column, nullptr, &numpy_array));
-
-  PyAcquireGIL lock;
-
-  for (int c = 0; c < data.num_chunks(); c++) {
-    auto arr = std::static_pointer_cast<ListArray>(data.chunk(c));
-
-    const uint8_t* data_ptr;
-    const bool has_nulls = data.null_count() > 0;
-    for (int64_t i = 0; i < arr->length(); ++i) {
-      if (has_nulls && arr->IsNull(i)) {
-        Py_INCREF(Py_None);
-        *out_values = Py_None;
-      } else {
-        PyObject* start = PyLong_FromLong(arr->value_offset(i));
-        PyObject* end = PyLong_FromLong(arr->value_offset(i + 1));
-        PyObject* slice = PySlice_New(start, end, NULL);
-        *out_values = PyObject_GetItem(numpy_array, slice);
-        Py_DECREF(start);
-        Py_DECREF(end);
-        Py_DECREF(slice);
-      }
-      ++out_values;
-    }
-  }
-
-  Py_XDECREF(numpy_array);
-  return Status::OK();
-}
-
-template <typename T>
-inline void ConvertNumericNullable(const ChunkedArray& data, T na_value, T* out_values) {
-  for (int c = 0; c < data.num_chunks(); c++) {
-    const std::shared_ptr<Array> arr = data.chunk(c);
-    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
-    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
-
-    const uint8_t* valid_bits = arr->null_bitmap_data();
-
-    if (arr->null_count() > 0) {
-      for (int64_t i = 0; i < arr->length(); ++i) {
-        *out_values++ = BitUtil::BitNotSet(valid_bits, i) ? na_value : in_values[i];
-      }
-    } else {
-      memcpy(out_values, in_values, sizeof(T) * arr->length());
-      out_values += arr->length();
-    }
-  }
-}
-
-template <typename InType, typename OutType>
-inline void ConvertNumericNullableCast(
-    const ChunkedArray& data, OutType na_value, OutType* out_values) {
-  for (int c = 0; c < data.num_chunks(); c++) {
-    const std::shared_ptr<Array> arr = data.chunk(c);
-    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
-    auto in_values = reinterpret_cast<const InType*>(prim_arr->data()->data());
-
-    for (int64_t i = 0; i < arr->length(); ++i) {
-      *out_values++ = arr->IsNull(i) ? na_value : static_cast<OutType>(in_values[i]);
-    }
-  }
-}
-
-template <typename T>
-inline void ConvertDates(const ChunkedArray& data, T na_value, T* out_values) {
-  for (int c = 0; c < data.num_chunks(); c++) {
-    const std::shared_ptr<Array> arr = data.chunk(c);
-    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
-    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
-
-    for (int64_t i = 0; i < arr->length(); ++i) {
-      // There are 1000 * 60 * 60 * 24 = 86400000ms in a day
-      *out_values++ = arr->IsNull(i) ? na_value : in_values[i] / 86400000;
-    }
-  }
-}
-
-template <typename InType, int SHIFT>
-inline void ConvertDatetimeNanos(const ChunkedArray& data, int64_t* out_values) {
-  for (int c = 0; c < data.num_chunks(); c++) {
-    const std::shared_ptr<Array> arr = data.chunk(c);
-    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
-    auto in_values = reinterpret_cast<const InType*>(prim_arr->data()->data());
-
-    for (int64_t i = 0; i < arr->length(); ++i) {
-      *out_values++ = arr->IsNull(i) ? kPandasTimestampNull
-                                     : (static_cast<int64_t>(in_values[i]) * SHIFT);
-    }
-  }
-}
-
-#define CONVERTLISTSLIKE_CASE(ArrowType, ArrowEnum)                \
-  case Type::ArrowEnum:                                            \
-    RETURN_NOT_OK((ConvertListsLike<ArrowType>(col, out_buffer))); \
-    break;
-
-class ObjectBlock : public PandasBlock {
- public:
-  using PandasBlock::PandasBlock;
-  virtual ~ObjectBlock() {}
-
-  Status Allocate() override { return AllocateNDArray(NPY_OBJECT); }
-
-  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
-      int64_t rel_placement) override {
-    Type::type type = col->type()->type;
-
-    PyObject** out_buffer =
-        reinterpret_cast<PyObject**>(block_data_) + rel_placement * num_rows_;
-
-    const ChunkedArray& data = *col->data().get();
-
-    if (type == Type::BOOL) {
-      RETURN_NOT_OK(ConvertBooleanWithNulls(data, out_buffer));
-    } else if (type == Type::BINARY) {
-      RETURN_NOT_OK(ConvertBinaryLike<BinaryArray>(data, out_buffer));
-    } else if (type == Type::STRING) {
-      RETURN_NOT_OK(ConvertBinaryLike<StringArray>(data, out_buffer));
-    } else if (type == Type::LIST) {
-      auto list_type = std::static_pointer_cast<ListType>(col->type());
-      switch (list_type->value_type()->type) {
-        CONVERTLISTSLIKE_CASE(UInt8Type, UINT8)
-        CONVERTLISTSLIKE_CASE(Int8Type, INT8)
-        CONVERTLISTSLIKE_CASE(UInt16Type, UINT16)
-        CONVERTLISTSLIKE_CASE(Int16Type, INT16)
-        CONVERTLISTSLIKE_CASE(UInt32Type, UINT32)
-        CONVERTLISTSLIKE_CASE(Int32Type, INT32)
-        CONVERTLISTSLIKE_CASE(UInt64Type, UINT64)
-        CONVERTLISTSLIKE_CASE(Int64Type, INT64)
-        CONVERTLISTSLIKE_CASE(TimestampType, TIMESTAMP)
-        CONVERTLISTSLIKE_CASE(FloatType, FLOAT)
-        CONVERTLISTSLIKE_CASE(DoubleType, DOUBLE)
-        CONVERTLISTSLIKE_CASE(StringType, STRING)
-        default: {
-          std::stringstream ss;
-          ss << "Not implemented type for lists: " << list_type->value_type()->ToString();
-          return Status::NotImplemented(ss.str());
-        }
-      }
-    } else {
-      std::stringstream ss;
-      ss << "Unsupported type for object array output: " << col->type()->ToString();
-      return Status::NotImplemented(ss.str());
-    }
-
-    placement_data_[rel_placement] = abs_placement;
-    return Status::OK();
-  }
-};
-
-template <int ARROW_TYPE, typename C_TYPE>
-class IntBlock : public PandasBlock {
- public:
-  using PandasBlock::PandasBlock;
-
-  Status Allocate() override {
-    return AllocateNDArray(arrow_traits<ARROW_TYPE>::npy_type);
-  }
-
-  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
-      int64_t rel_placement) override {
-    Type::type type = col->type()->type;
-
-    C_TYPE* out_buffer =
-        reinterpret_cast<C_TYPE*>(block_data_) + rel_placement * num_rows_;
-
-    const ChunkedArray& data = *col->data().get();
-
-    if (type != ARROW_TYPE) { return Status::NotImplemented(col->type()->ToString()); }
-
-    ConvertIntegerNoNullsSameType<C_TYPE>(data, out_buffer);
-    placement_data_[rel_placement] = abs_placement;
-    return Status::OK();
-  }
-};
-
-using UInt8Block = IntBlock<Type::UINT8, uint8_t>;
-using Int8Block = IntBlock<Type::INT8, int8_t>;
-using UInt16Block = IntBlock<Type::UINT16, uint16_t>;
-using Int16Block = IntBlock<Type::INT16, int16_t>;
-using UInt32Block = IntBlock<Type::UINT32, uint32_t>;
-using Int32Block = IntBlock<Type::INT32, int32_t>;
-using UInt64Block = IntBlock<Type::UINT64, uint64_t>;
-using Int64Block = IntBlock<Type::INT64, int64_t>;
-
-class Float32Block : public PandasBlock {
- public:
-  using PandasBlock::PandasBlock;
-
-  Status Allocate() override { return AllocateNDArray(NPY_FLOAT32); }
-
-  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
-      int64_t rel_placement) override {
-    Type::type type = col->type()->type;
-
-    if (type != Type::FLOAT) { return Status::NotImplemented(col->type()->ToString()); }
-
-    float* out_buffer = reinterpret_cast<float*>(block_data_) + rel_placement * num_rows_;
-
-    ConvertNumericNullable<float>(*col->data().get(), NAN, out_buffer);
-    placement_data_[rel_placement] = abs_placement;
-    return Status::OK();
-  }
-};
-
-class Float64Block : public PandasBlock {
- public:
-  using PandasBlock::PandasBlock;
-
-  Status Allocate() override { return AllocateNDArray(NPY_FLOAT64); }
-
-  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
-      int64_t rel_placement) override {
-    Type::type type = col->type()->type;
-
-    double* out_buffer =
-        reinterpret_cast<double*>(block_data_) + rel_placement * num_rows_;
-
-    const ChunkedArray& data = *col->data().get();
-
-#define INTEGER_CASE(IN_TYPE)                         \
-  ConvertIntegerWithNulls<IN_TYPE>(data, out_buffer); \
-  break;
-
-    switch (type) {
-      case Type::UINT8:
-        INTEGER_CASE(uint8_t);
-      case Type::INT8:
-        INTEGER_CASE(int8_t);
-      case Type::UINT16:
-        INTEGER_CASE(uint16_t);
-      case Type::INT16:
-        INTEGER_CASE(int16_t);
-      case Type::UINT32:
-        INTEGER_CASE(uint32_t);
-      case Type::INT32:
-        INTEGER_CASE(int32_t);
-      case Type::UINT64:
-        INTEGER_CASE(uint64_t);
-      case Type::INT64:
-        INTEGER_CASE(int64_t);
-      case Type::FLOAT:
-        ConvertNumericNullableCast<float, double>(data, NAN, out_buffer);
-        break;
-      case Type::DOUBLE:
-        ConvertNumericNullable<double>(data, NAN, out_buffer);
-        break;
-      default:
-        return Status::NotImplemented(col->type()->ToString());
-    }
-
-#undef INTEGER_CASE
-
-    placement_data_[rel_placement] = abs_placement;
-    return Status::OK();
-  }
-};
-
-class BoolBlock : public PandasBlock {
- public:
-  using PandasBlock::PandasBlock;
-
-  Status Allocate() override { return AllocateNDArray(NPY_BOOL); }
-
-  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
-      int64_t rel_placement) override {
-    Type::type type = col->type()->type;
-
-    if (type != Type::BOOL) { return Status::NotImplemented(col->type()->ToString()); }
-
-    uint8_t* out_buffer =
-        reinterpret_cast<uint8_t*>(block_data_) + rel_placement * num_rows_;
-
-    ConvertBooleanNoNulls(*col->data().get(), out_buffer);
-    placement_data_[rel_placement] = abs_placement;
-    return Status::OK();
-  }
-};
-
-class DatetimeBlock : public PandasBlock {
- public:
-  using PandasBlock::PandasBlock;
-
-  Status AllocateDatetime(int ndim) {
-    RETURN_NOT_OK(AllocateNDArray(NPY_DATETIME, ndim));
-
-    PyAcquireGIL lock;
-    auto date_dtype = reinterpret_cast<PyArray_DatetimeDTypeMetaData*>(
-        PyArray_DESCR(reinterpret_cast<PyArrayObject*>(block_arr_.obj()))->c_metadata);
-    date_dtype->meta.base = NPY_FR_ns;
-    return Status::OK();
-  }
-
-  Status Allocate() override { return AllocateDatetime(2); }
-
-  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
-      int64_t rel_placement) override {
-    Type::type type = col->type()->type;
-
-    int64_t* out_buffer =
-        reinterpret_cast<int64_t*>(block_data_) + rel_placement * num_rows_;
-
-    const ChunkedArray& data = *col.get()->data();
-
-    if (type == Type::DATE64) {
-      // Date64Type is millisecond timestamp stored as int64_t
-      // TODO(wesm): Do we want to make sure to zero out the milliseconds?
-      ConvertDatetimeNanos<int64_t, 1000000L>(data, out_buffer);
-    } else if (type == Type::TIMESTAMP) {
-      auto ts_type = static_cast<TimestampType*>(col->type().get());
-
-      if (ts_type->unit == TimeUnit::NANO) {
-        ConvertNumericNullable<int64_t>(data, kPandasTimestampNull, out_buffer);
-      } else if (ts_type->unit == TimeUnit::MICRO) {
-        ConvertDatetimeNanos<int64_t, 1000L>(data, out_buffer);
-      } else if (ts_type->unit == TimeUnit::MILLI) {
-        ConvertDatetimeNanos<int64_t, 1000000L>(data, out_buffer);
-      } else if (ts_type->unit == TimeUnit::SECOND) {
-        ConvertDatetimeNanos<int64_t, 1000000000L>(data, out_buffer);
-      } else {
-        return Status::NotImplemented("Unsupported time unit");
-      }
-    } else {
-      return Status::NotImplemented(col->type()->ToString());
-    }
-
-    placement_data_[rel_placement] = abs_placement;
-    return Status::OK();
-  }
-};
-
-class DatetimeTZBlock : public DatetimeBlock {
- public:
-  DatetimeTZBlock(const std::string& timezone, int64_t num_rows)
-      : DatetimeBlock(num_rows, 1), timezone_(timezone) {}
-
-  // Like Categorical, the internal ndarray is 1-dimensional
-  Status Allocate() override { return AllocateDatetime(1); }
-
-  Status GetPyResult(PyObject** output) override {
-    PyObject* result = PyDict_New();
-    RETURN_IF_PYERROR();
-
-    PyObject* py_tz = PyUnicode_FromStringAndSize(
-        timezone_.c_str(), static_cast<Py_ssize_t>(timezone_.size()));
-    RETURN_IF_PYERROR();
-
-    PyDict_SetItemString(result, "block", block_arr_.obj());
-    PyDict_SetItemString(result, "timezone", py_tz);
-    PyDict_SetItemString(result, "placement", placement_arr_.obj());
-
-    *output = result;
-
-    return Status::OK();
-  }
-
- private:
-  std::string timezone_;
-};
-
-template <int ARROW_INDEX_TYPE>
-class CategoricalBlock : public PandasBlock {
- public:
-  CategoricalBlock(int64_t num_rows) : PandasBlock(num_rows, 1) {}
-
-  Status Allocate() override {
-    constexpr int npy_type = arrow_traits<ARROW_INDEX_TYPE>::npy_type;
-
-    if (!(npy_type == NPY_INT8 || npy_type == NPY_INT16 || npy_type == NPY_INT32 ||
-            npy_type == NPY_INT64)) {
-      return Status::Invalid("Category indices must be signed integers");
-    }
-    return AllocateNDArray(npy_type, 1);
-  }
-
-  Status Write(const std::shared_ptr<Column>& col, int64_t abs_placement,
-      int64_t rel_placement) override {
-    using T = typename arrow_traits<ARROW_INDEX_TYPE>::T;
-
-    T* out_values = reinterpret_cast<T*>(block_data_) + rel_placement * num_rows_;
-
-    const ChunkedArray& data = *col->data().get();
-
-    for (int c = 0; c < data.num_chunks(); c++) {
-      const std::shared_ptr<Array> arr = data.chunk(c);
-      const auto& dict_arr = static_cast<const DictionaryArray&>(*arr);
-      const auto& indices = static_cast<const PrimitiveArray&>(*dict_arr.indices());
-      auto in_values = reinterpret_cast<const T*>(indices.data()->data());
-
-      // Null is -1 in CategoricalBlock
-      for (int i = 0; i < arr->length(); ++i) {
-        *out_values++ = indices.IsNull(i) ? -1 : in_values[i];
-      }
-    }
-
-    placement_data_[rel_placement] = abs_placement;
-
-    auto dict_type = static_cast<const DictionaryType*>(col->type().get());
-
-    PyObject* dict;
-    RETURN_NOT_OK(ConvertArrayToPandas(dict_type->dictionary(), nullptr, &dict));
-    dictionary_.reset(dict);
-
-    return Status::OK();
-  }
-
-  Status GetPyResult(PyObject** output) override {
-    PyObject* result = PyDict_New();
-    RETURN_IF_PYERROR();
-
-    PyDict_SetItemString(result, "block", block_arr_.obj());
-    PyDict_SetItemString(result, "dictionary", dictionary_.obj());
-    PyDict_SetItemString(result, "placement", placement_arr_.obj());
-
-    *output = result;
-
-    return Status::OK();
-  }
-
- protected:
-  OwnedRef dictionary_;
-};
-
-Status MakeBlock(PandasBlock::type type, int64_t num_rows, int num_columns,
-    std::shared_ptr<PandasBlock>* block) {
-#define BLOCK_CASE(NAME, TYPE)                              \
-  case PandasBlock::NAME:                                   \
-    *block = std::make_shared<TYPE>(num_rows, num_columns); \
-    break;
-
-  switch (type) {
-    BLOCK_CASE(OBJECT, ObjectBlock);
-    BLOCK_CASE(UINT8, UInt8Block);
-    BLOCK_CASE(INT8, Int8Block);
-    BLOCK_CASE(UINT16, UInt16Block);
-    BLOCK_CASE(INT16, Int16Block);
-    BLOCK_CASE(UINT32, UInt32Block);
-    BLOCK_CASE(INT32, Int32Block);
-    BLOCK_CASE(UINT64, UInt64Block);
-    BLOCK_CASE(INT64, Int64Block);
-    BLOCK_CASE(FLOAT, Float32Block);
-    BLOCK_CASE(DOUBLE, Float64Block);
-    BLOCK_CASE(BOOL, BoolBlock);
-    BLOCK_CASE(DATETIME, DatetimeBlock);
-    default:
-      return Status::NotImplemented("Unsupported block type");
-  }
-
-#undef BLOCK_CASE
-
-  return (*block)->Allocate();
-}
-
-static inline Status MakeCategoricalBlock(const std::shared_ptr<DataType>& type,
-    int64_t num_rows, std::shared_ptr<PandasBlock>* block) {
-  // All categoricals become a block with a single column
-  auto dict_type = static_cast<const DictionaryType*>(type.get());
-  switch (dict_type->index_type()->type) {
-    case Type::INT8:
-      *block = std::make_shared<CategoricalBlock<Type::INT8>>(num_rows);
-      break;
-    case Type::INT16:
-      *block = std::make_shared<CategoricalBlock<Type::INT16>>(num_rows);
-      break;
-    case Type::INT32:
-      *block = std::make_shared<CategoricalBlock<Type::INT32>>(num_rows);
-      break;
-    case Type::INT64:
-      *block = std::make_shared<CategoricalBlock<Type::INT64>>(num_rows);
-      break;
-    default: {
-      std::stringstream ss;
-      ss << "Categorical index type not implemented: "
-         << dict_type->index_type()->ToString();
-      return Status::NotImplemented(ss.str());
-    }
-  }
-  return (*block)->Allocate();
-}
-
-using BlockMap = std::unordered_map<int, std::shared_ptr<PandasBlock>>;
-
-// Construct the exact pandas 0.x "BlockManager" memory layout
-//
-// * For each column determine the correct output pandas type
-// * Allocate 2D blocks (ncols x nrows) for each distinct data type in output
-// * Allocate  block placement arrays
-// * Write Arrow columns out into each slice of memory; populate block
-// * placement arrays as we go
-class DataFrameBlockCreator {
- public:
-  DataFrameBlockCreator(const std::shared_ptr<Table>& table) : table_(table) {}
-
-  Status Convert(int nthreads, PyObject** output) {
-    column_types_.resize(table_->num_columns());
-    column_block_placement_.resize(table_->num_columns());
-    type_counts_.clear();
-    blocks_.clear();
-
-    RETURN_NOT_OK(CreateBlocks());
-    RETURN_NOT_OK(WriteTableToBlocks(nthreads));
-
-    return GetResultList(output);
-  }
-
-  Status CreateBlocks() {
-    for (int i = 0; i < table_->num_columns(); ++i) {
-      std::shared_ptr<Column> col = table_->column(i);
-      PandasBlock::type output_type;
-
-      Type::type column_type = col->type()->type;
-      switch (column_type) {
-        case Type::BOOL:
-          output_type = col->null_count() > 0 ? PandasBlock::OBJECT : PandasBlock::BOOL;
-          break;
-        case Type::UINT8:
-          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::UINT8;
-          break;
-        case Type::INT8:
-          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::INT8;
-          break;
-        case Type::UINT16:
-          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::UINT16;
-          break;
-        case Type::INT16:
-          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::INT16;
-          break;
-        case Type::UINT32:
-          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::UINT32;
-          break;
-        case Type::INT32:
-          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::INT32;
-          break;
-        case Type::INT64:
-          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::INT64;
-          break;
-        case Type::UINT64:
-          output_type = col->null_count() > 0 ? PandasBlock::DOUBLE : PandasBlock::UINT64;
-          break;
-        case Type::FLOAT:
-          output_type = PandasBlock::FLOAT;
-          break;
-        case Type::DOUBLE:
-          output_type = PandasBlock::DOUBLE;
-          break;
-        case Type::STRING:
-        case Type::BINARY:
-          output_type = PandasBlock::OBJECT;
-          break;
-        case Type::DATE64:
-          output_type = PandasBlock::DATETIME;
-          break;
-        case Type::TIMESTAMP: {
-          const auto& ts_type = static_cast<const TimestampType&>(*col->type());
-          if (ts_type.timezone != "") {
-            output_type = PandasBlock::DATETIME_WITH_TZ;
-          } else {
-            output_type = PandasBlock::DATETIME;
-          }
-        } break;
-        case Type::LIST: {
-          auto list_type = std::static_pointer_cast<ListType>(col->type());
-          if (!ListTypeSupported(list_type->value_type()->type)) {
-            std::stringstream ss;
-            ss << "Not implemented type for lists: "
-               << list_type->value_type()->ToString();
-            return Status::NotImplemented(ss.str());
-          }
-          output_type = PandasBlock::OBJECT;
-        } break;
-        case Type::DICTIONARY:
-          output_type = PandasBlock::CATEGORICAL;
-          break;
-        default:
-          return Status::NotImplemented(col->type()->ToString());
-      }
-
-      int block_placement = 0;
-      std::shared_ptr<PandasBlock> block;
-      if (output_type == PandasBlock::CATEGORICAL) {
-        RETURN_NOT_OK(MakeCategoricalBlock(col->type(), table_->num_rows(), &block));
-        categorical_blocks_[i] = block;
-      } else if (output_type == PandasBlock::DATETIME_WITH_TZ) {
-        const auto& ts_type = static_cast<const TimestampType&>(*col->type());
-        block = std::make_shared<DatetimeTZBlock>(ts_type.timezone, table_->num_rows());
-        RETURN_NOT_OK(block->Allocate());
-        datetimetz_blocks_[i] = block;
-      } else {
-        auto it = type_counts_.find(output_type);
-        if (it != type_counts_.end()) {
-          block_placement = it->second;
-          // Increment count
-          it->second += 1;
-        } else {
-          // Add key to map
-          type_counts_[output_type] = 1;
-        }
-      }
-
-      column_types_[i] = output_type;
-      column_block_placement_[i] = block_placement;
-    }
-
-    // Create normal non-categorical blocks
-    for (const auto& it : type_counts_) {
-      PandasBlock::type type = static_cast<PandasBlock::type>(it.first);
-      std::shared_ptr<PandasBlock> block;
-      RETURN_NOT_OK(MakeBlock(type, table_->num_rows(), it.second, &block));
-      blocks_[type] = block;
-    }
-    return Status::OK();
-  }
-
-  Status WriteTableToBlocks(int nthreads) {
-    auto WriteColumn = [this](int i) {
-      std::shared_ptr<Column> col = this->table_->column(i);
-      PandasBlock::type output_type = this->column_types_[i];
-
-      int rel_placement = this->column_block_placement_[i];
-
-      std::shared_ptr<PandasBlock> block;
-      if (output_type == PandasBlock::CATEGORICAL) {
-        auto it = this->categorical_blocks_.find(i);
-        if (it == this->blocks_.end()) {
-          return Status::KeyError("No categorical block allocated");
-        }
-        block = it->second;
-      } else if (output_type == PandasBlock::DATETIME_WITH_TZ) {
-        auto it = this->datetimetz_blocks_.find(i);
-        if (it == this->datetimetz_blocks_.end()) {
-          return Status::KeyError("No datetimetz block allocated");
-        }
-        block = it->second;
-      } else {
-        auto it = this->blocks_.find(output_type);
-        if (it == this->blocks_.end()) { return Status::KeyError("No block allocated"); }
-        block = it->second;
-      }
-      return block->Write(col, i, rel_placement);
-    };
-
-    nthreads = std::min<int>(nthreads, table_->num_columns());
-
-    if (nthreads == 1) {
-      for (int i = 0; i < table_->num_columns(); ++i) {
-        RETURN_NOT_OK(WriteColumn(i));
-      }
-    } else {
-      std::vector<std::thread> thread_pool;
-      thread_pool.reserve(nthreads);
-      std::atomic<int> task_counter(0);
-
-      std::mutex error_mtx;
-      bool error_occurred = false;
-      Status error;
-
-      for (int thread_id = 0; thread_id < nthreads; ++thread_id) {
-        thread_pool.emplace_back(
-            [this, &error, &error_occurred, &error_mtx, &task_counter, &WriteColumn]() {
-              int column_num;
-              while (!error_occurred) {
-                column_num = task_counter.fetch_add(1);
-                if (column_num >= this->table_->num_columns()) { break; }
-                Status s = WriteColumn(column_num);
-                if (!s.ok()) {
-                  std::lock_guard<std::mutex> lock(error_mtx);
-                  error_occurred = true;
-                  error = s;
-                  break;
-                }
-              }
-            });
-      }
-      for (auto&& thread : thread_pool) {
-        thread.join();
-      }
-
-      if (error_occurred) { return error; }
-    }
-    return Status::OK();
-  }
-
-  Status AppendBlocks(const BlockMap& blocks, PyObject* list) {
-    for (const auto& it : blocks) {
-      PyObject* item;
-      RETURN_NOT_OK(it.second->GetPyResult(&item));
-      if (PyList_Append(list, item) < 0) { RETURN_IF_PYERROR(); }
-    }
-    return Status::OK();
-  }
-
-  Status GetResultList(PyObject** out) {
-    PyAcquireGIL lock;
-
-    PyObject* result = PyList_New(0);
-    RETURN_IF_PYERROR();
-
-    RETURN_NOT_OK(AppendBlocks(blocks_, result));
-    RETURN_NOT_OK(AppendBlocks(categorical_blocks_, result));
-    RETURN_NOT_OK(AppendBlocks(datetimetz_blocks_, result));
-
-    *out = result;
-    return Status::OK();
-  }
-
- private:
-  std::shared_ptr<Table> table_;
-
-  // column num -> block type id
-  std::vector<PandasBlock::type> column_types_;
-
-  // column num -> relative placement within internal block
-  std::vector<int> column_block_placement_;
-
-  // block type -> type count
-  std::unordered_map<int, int> type_counts_;
-
-  // block type -> block
-  BlockMap blocks_;
-
-  // column number -> categorical block
-  BlockMap categorical_blocks_;
-
-  // column number -> datetimetz block
-  BlockMap datetimetz_blocks_;
-};
-
-class ArrowDeserializer {
- public:
-  ArrowDeserializer(const std::shared_ptr<Column>& col, PyObject* py_ref)
-      : col_(col), data_(*col->data().get()), py_ref_(py_ref) {}
-
-  Status AllocateOutput(int type) {
-    PyAcquireGIL lock;
-
-    npy_intp dims[1] = {col_->length()};
-    result_ = PyArray_SimpleNew(1, dims, type);
-    arr_ = reinterpret_cast<PyArrayObject*>(result_);
-
-    if (arr_ == NULL) {
-      // Error occurred, trust that SimpleNew set the error state
-      return Status::OK();
-    }
-
-    set_numpy_metadata(type, col_->type().get(), arr_);
-
-    return Status::OK();
-  }
-
-  template <int TYPE>
-  Status ConvertValuesZeroCopy(int npy_type, std::shared_ptr<Array> arr) {
-    typedef typename arrow_traits<TYPE>::T T;
-
-    auto prim_arr = static_cast<PrimitiveArray*>(arr.get());
-    auto in_values = reinterpret_cast<const T*>(prim_arr->data()->data());
-
-    // Zero-Copy. We can pass the data pointer directly to NumPy.
-    void* data = const_cast<T*>(in_values);
-
-    PyAcquireGIL lock;
-
-    // Zero-Copy. We can pass the data pointer directly to NumPy.
-    npy_intp dims[1] = {col_->length()};
-    result_ = PyArray_SimpleNewFromData(1, dims, npy_type, data);
-    arr_ = reinterpret_cast<PyArrayObject*>(result_);
-
-    if (arr_ == NULL) {
-      // Error occurred, trust that SimpleNew set the error state
-      return Status::OK();
-    }
-
-    set_numpy_metadata(npy_type, col_->type().get(), arr_);
-
-    if (PyArray_SetBaseObject(arr_, py_ref_) == -1) {
-      // Error occurred, trust that SetBaseObject set the error state
-      return Status::OK();
-    } else {
-      // PyArray_SetBaseObject steals our reference to py_ref_
-      Py_INCREF(py_ref_);
-    }
-
-    // Arrow data is immutable.
-    PyArray_CLEARFLAGS(arr_, NPY_ARRAY_WRITEABLE);
-
-    return Status::OK();
-  }
-
-  // ----------------------------------------------------------------------
-  // Allocate new array and deserialize. Can do a zero copy conversion for some
-  // types
-
-  Status Convert(PyObject** out) {
-#define CONVERT_CASE(TYPE)                      \
-  case Type::TYPE: {                            \
-    RETURN_NOT_OK(ConvertValues<Type::TYPE>()); \
-  } break;
-
-    switch (col_->type()->type) {
-      CONVERT_CASE(BOOL);
-      CONVERT_CASE(INT8);
-      CONVERT_CASE(INT16);
-      CONVERT_CASE(INT32);
-      CONVERT_CASE(INT64);
-      CONVERT_CASE(UINT8);
-      CONVERT_CASE(UINT16);
-      CONVERT_CASE(UINT32);
-      CONVERT_CASE(UINT64);
-      CONVERT_CASE(FLOAT);
-      CONVERT_CASE(DOUBLE);
-      CONVERT_CASE(BINARY);
-      CONVERT_CASE(STRING);
-      CONVERT_CASE(DATE64);
-      CONVERT_CASE(TIMESTAMP);
-      CONVERT_CASE(DICTIONARY);
-      CONVERT_CASE(LIST);
-      default: {
-        std::stringstream ss;
-        ss << "Arrow type reading not implemented for " << col_->type()->ToString();
-        return Status::NotImplemented(ss.str());
-      }
-    }
-
-#undef CONVERT_CASE
-
-    *out = result_;
-    return Status::OK();
-  }
-
-  template <int TYPE>
-  inline typename std::enable_if<
-      (TYPE != Type::DATE64) & arrow_traits<TYPE>::is_numeric_nullable, Status>::type
-  ConvertValues() {
-    typedef typename arrow_traits<TYPE>::T T;
-    int npy_type = arrow_traits<TYPE>::npy_type;
-
-    if (data_.num_chunks() == 1 && data_.null_count() == 0 && py_ref_ != nullptr) {
-      return ConvertValuesZeroCopy<TYPE>(npy_type, data_.chunk(0));
-    }
-
-    RETURN_NOT_OK(AllocateOutput(npy_type));
-    auto out_values = reinterpret_cast<T*>(PyArray_DATA(arr_));
-    ConvertNumericNullable<T>(data_, arrow_traits<TYPE>::na_value, out_values);
-
-    return Status::OK();
-  }
-
-  template <int TYPE>
-  inline typename std::enable_if<TYPE == Type::DATE64, Status>::type ConvertValues() {
-    typedef typename arrow_traits<TYPE>::T T;
-
-    RETURN_NOT_OK(AllocateOutput(arrow_traits<TYPE>::npy_type));
-    auto out_values = reinterpret_cast<T*>(PyArray_DATA(arr_));
-    ConvertDates<T>(data_, arrow_traits<TYPE>::na_value, out_values);
-    return Status::OK();
-  }
-
-  // Integer specialization
-  template <int TYPE>
-  inline
-      typename std::enable_if<arrow_traits<TYPE>::is_numeric_not_nullable, Status>::type
-      ConvertValues() {
-    typedef typename arrow_traits<TYPE>::T T;
-    int npy_type = arrow_traits<TYPE>::npy_type;
-
-    if (data_.num_chunks() == 1 && data_.null_count() == 0 && py_ref_ != nullptr) {
-      return ConvertValuesZeroCopy<TYPE>(npy_type, data_.chunk(0));
-    }
-
-    if (data_.null_count() > 0) {
-      RETURN_NOT_OK(AllocateOutput(NPY_FLOAT64));
-      auto out_values = reinterpret_cast<double*>(PyArray_DATA(arr_));
-      ConvertIntegerWithNulls<T>(data_, out_values);
-    } else {
-      RETURN_NOT_OK(AllocateOutput(arrow_traits<TYPE>::npy_type));
-      auto out_values = reinterpret_cast<T*>(PyArray_DATA(arr_));
-      ConvertIntegerNoNullsSameType<T>(data_, out_values);
-    }
-
-    return Status::OK();
-  }
-
-  // Boolean specialization
-  template <int TYPE>
-  inline typename std::enable_if<arrow_traits<TYPE>::is_boolean, Status>::type
-  ConvertValues() {
-    if (data_.null_count() > 0) {
-      RETURN_NOT_OK(AllocateOutput(NPY_OBJECT));
-      auto out_values = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-      RETURN_NOT_OK(ConvertBooleanWithNulls(data_, out_values));
-    } else {
-      RETURN_NOT_OK(AllocateOutput(arrow_traits<TYPE>::npy_type));
-      auto out_values = reinterpret_cast<uint8_t*>(PyArray_DATA(arr_));
-      ConvertBooleanNoNulls(data_, out_values);
-    }
-    return Status::OK();
-  }
-
-  // UTF8 strings
-  template <int TYPE>
-  inline typename std::enable_if<TYPE == Type::STRING, Status>::type ConvertValues() {
-    RETURN_NOT_OK(AllocateOutput(NPY_OBJECT));
-    auto out_values = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-    return ConvertBinaryLike<StringArray>(data_, out_values);
-  }
-
-  template <int T2>
-  inline typename std::enable_if<T2 == Type::BINARY, Status>::type ConvertValues() {
-    RETURN_NOT_OK(AllocateOutput(NPY_OBJECT));
-    auto out_values = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-    return ConvertBinaryLike<BinaryArray>(data_, out_values);
-  }
-
-#define CONVERTVALUES_LISTSLIKE_CASE(ArrowType, ArrowEnum) \
-  case Type::ArrowEnum:                                    \
-    return ConvertListsLike<ArrowType>(col_, out_values);
-
-  template <int T2>
-  inline typename std::enable_if<T2 == Type::LIST, Status>::type ConvertValues() {
-    RETURN_NOT_OK(AllocateOutput(NPY_OBJECT));
-    auto out_values = reinterpret_cast<PyObject**>(PyArray_DATA(arr_));
-    auto list_type = std::static_pointer_cast<ListType>(col_->type());
-    switch (list_type->value_type()->type) {
-      CONVERTVALUES_LISTSLIKE_CASE(UInt8Type, UINT8)
-      CONVERTVALUES_LISTSLIKE_CASE(Int8Type, INT8)
-      CONVERTVALUES_LISTSLIKE_CASE(UInt16Type, UINT16)
-      CONVERTVALUES_LISTSLIKE_CASE(Int16Type, INT16)
-      CONVERTVALUES_LISTSLIKE_CASE(UInt32Type, UINT32)
-      CONVERTVALUES_LISTSLIKE_CASE(Int32Type, INT32)
-      CONVERTVALUES_LISTSLIKE_CASE(UInt64Type, UINT64)
-      CONVERTVALUES_LISTSLIKE_CASE(Int64Type, INT64)
-      CONVERTVALUES_LISTSLIKE_CASE(TimestampType, TIMESTAMP)
-      CONVERTVALUES_LISTSLIKE_CASE(FloatType, FLOAT)
-      CONVERTVALUES_LISTSLIKE_CASE(DoubleType, DOUBLE)
-      CONVERTVALUES_LISTSLIKE_CASE(StringType, STRING)
-      default: {
-        std::stringstream ss;
-        ss << "Not implemented type for lists: " << list_type->value_type()->ToString();
-        return Status::NotImplemented(ss.str());
-      }
-    }
-  }
-
-  template <int TYPE>
-  inline typename std::enable_if<TYPE == Type::DICTIONARY, Status>::type ConvertValues() {
-    std::shared_ptr<PandasBlock> block;
-    RETURN_NOT_OK(MakeCategoricalBlock(col_->type(), col_->length(), &block));
-    RETURN_NOT_OK(block->Write(col_, 0, 0));
-
-    auto dict_type = static_cast<const DictionaryType*>(col_->type().get());
-
-    PyAcquireGIL lock;
-    result_ = PyDict_New();
-    RETURN_IF_PYERROR();
-
-    PyObject* dictionary;
-    RETURN_NOT_OK(ConvertArrayToPandas(dict_type->dictionary(), nullptr, &dictionary));
-
-    PyDict_SetItemString(result_, "indices", block->block_arr());
-    PyDict_SetItemString(result_, "dictionary", dictionary);
-
-    return Status::OK();
-  }
-
- private:
-  std::shared_ptr<Column> col_;
-  const ChunkedArray& data_;
-  PyObject* py_ref_;
-  PyArrayObject* arr_;
-  PyObject* result_;
-};
-
-Status ConvertArrayToPandas(
-    const std::shared_ptr<Array>& arr, PyObject* py_ref, PyObject** out) {
-  static std::string dummy_name = "dummy";
-  auto field = std::make_shared<Field>(dummy_name, arr->type());
-  auto col = std::make_shared<Column>(field, arr);
-  return ConvertColumnToPandas(col, py_ref, out);
-}
-
-Status ConvertColumnToPandas(
-    const std::shared_ptr<Column>& col, PyObject* py_ref, PyObject** out) {
-  ArrowDeserializer converter(col, py_ref);
-  return converter.Convert(out);
-}
-
-Status ConvertTableToPandas(
-    const std::shared_ptr<Table>& table, int nthreads, PyObject** out) {
-  DataFrameBlockCreator helper(table);
-  return helper.Convert(nthreads, out);
-}
-
-}  // namespace py
-}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/adapters/pandas.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/pandas.h b/python/src/pyarrow/adapters/pandas.h
deleted file mode 100644
index 6862339..0000000
--- a/python/src/pyarrow/adapters/pandas.h
+++ /dev/null
@@ -1,79 +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.
-
-// Functions for converting between pandas's NumPy-based data representation
-// and Arrow data structures
-
-#ifndef PYARROW_ADAPTERS_PANDAS_H
-#define PYARROW_ADAPTERS_PANDAS_H
-
-#include <Python.h>
-
-#include <memory>
-
-#include "arrow/util/visibility.h"
-
-namespace arrow {
-
-class Array;
-class Column;
-class DataType;
-class MemoryPool;
-class Status;
-class Table;
-
-namespace py {
-
-ARROW_EXPORT
-Status ConvertArrayToPandas(
-    const std::shared_ptr<Array>& arr, PyObject* py_ref, PyObject** out);
-
-ARROW_EXPORT
-Status ConvertColumnToPandas(
-    const std::shared_ptr<Column>& col, PyObject* py_ref, PyObject** out);
-
-struct PandasOptions {
-  bool strings_to_categorical;
-};
-
-// Convert a whole table as efficiently as possible to a pandas.DataFrame.
-//
-// The returned Python object is a list of tuples consisting of the exact 2D
-// BlockManager structure of the pandas.DataFrame used as of pandas 0.19.x.
-//
-// tuple item: (indices: ndarray[int32], block: ndarray[TYPE, ndim=2])
-ARROW_EXPORT
-Status ConvertTableToPandas(
-    const std::shared_ptr<Table>& table, int nthreads, PyObject** out);
-
-ARROW_EXPORT
-Status PandasDtypeToArrow(PyObject* dtype, std::shared_ptr<DataType>* out);
-
-ARROW_EXPORT
-Status PandasToArrow(MemoryPool* pool, PyObject* ao, PyObject* mo,
-    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out);
-
-/// Convert dtype=object arrays. If target data type is not known, pass a type
-/// with nullptr
-ARROW_EXPORT
-Status PandasObjectsToArrow(MemoryPool* pool, PyObject* ao, PyObject* mo,
-    const std::shared_ptr<DataType>& type, std::shared_ptr<Array>* out);
-
-}  // namespace py
-}  // namespace arrow
-
-#endif  // PYARROW_ADAPTERS_PANDAS_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/api.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/api.h b/python/src/pyarrow/api.h
deleted file mode 100644
index f65cc09..0000000
--- a/python/src/pyarrow/api.h
+++ /dev/null
@@ -1,26 +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.
-
-#ifndef PYARROW_API_H
-#define PYARROW_API_H
-
-#include "pyarrow/helpers.h"
-
-#include "pyarrow/adapters/builtin.h"
-#include "pyarrow/adapters/pandas.h"
-
-#endif  // PYARROW_API_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/common.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/common.cc b/python/src/pyarrow/common.cc
deleted file mode 100644
index 792aa47..0000000
--- a/python/src/pyarrow/common.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 "pyarrow/common.h"
-
-#include <cstdlib>
-#include <mutex>
-#include <sstream>
-
-#include "arrow/memory_pool.h"
-#include "arrow/status.h"
-
-namespace arrow {
-namespace py {
-
-static std::mutex memory_pool_mutex;
-static MemoryPool* default_pyarrow_pool = nullptr;
-
-void set_default_memory_pool(MemoryPool* pool) {
-  std::lock_guard<std::mutex> guard(memory_pool_mutex);
-  default_pyarrow_pool = pool;
-}
-
-MemoryPool* get_memory_pool() {
-  std::lock_guard<std::mutex> guard(memory_pool_mutex);
-  if (default_pyarrow_pool) {
-    return default_pyarrow_pool;
-  } else {
-    return default_memory_pool();
-  }
-}
-
-// ----------------------------------------------------------------------
-// PyBuffer
-
-PyBuffer::PyBuffer(PyObject* obj)
-    : Buffer(nullptr, 0) {
-    if (PyObject_CheckBuffer(obj)) {
-        obj_ = PyMemoryView_FromObject(obj);
-        Py_buffer* buffer = PyMemoryView_GET_BUFFER(obj_);
-        data_ = reinterpret_cast<const uint8_t*>(buffer->buf);
-        size_ = buffer->len;
-        capacity_ = buffer->len;
-        is_mutable_ = false;
-        Py_INCREF(obj_);
-    } 
-}
-
-PyBuffer::~PyBuffer() {
-    PyAcquireGIL lock;
-    Py_DECREF(obj_);
-}
-
-}  // namespace py
-}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/common.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/common.h b/python/src/pyarrow/common.h
deleted file mode 100644
index b4e4ea6..0000000
--- a/python/src/pyarrow/common.h
+++ /dev/null
@@ -1,137 +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.
-
-#ifndef PYARROW_COMMON_H
-#define PYARROW_COMMON_H
-
-#include "pyarrow/config.h"
-
-#include "arrow/buffer.h"
-#include "arrow/util/macros.h"
-#include "arrow/util/visibility.h"
-
-namespace arrow {
-
-class MemoryPool;
-
-namespace py {
-
-class PyAcquireGIL {
- public:
-  PyAcquireGIL() { state_ = PyGILState_Ensure(); }
-
-  ~PyAcquireGIL() { PyGILState_Release(state_); }
-
- private:
-  PyGILState_STATE state_;
-  DISALLOW_COPY_AND_ASSIGN(PyAcquireGIL);
-};
-
-#define PYARROW_IS_PY2 PY_MAJOR_VERSION <= 2
-
-class OwnedRef {
- public:
-  OwnedRef() : obj_(nullptr) {}
-
-  OwnedRef(PyObject* obj) : obj_(obj) {}
-
-  ~OwnedRef() {
-    PyAcquireGIL lock;
-    Py_XDECREF(obj_);
-  }
-
-  void reset(PyObject* obj) {
-    if (obj_ != nullptr) { Py_XDECREF(obj_); }
-    obj_ = obj;
-  }
-
-  void release() { obj_ = nullptr; }
-
-  PyObject* obj() const { return obj_; }
-
- private:
-  PyObject* obj_;
-};
-
-struct PyObjectStringify {
-  OwnedRef tmp_obj;
-  const char* bytes;
-
-  PyObjectStringify(PyObject* obj) {
-    PyObject* bytes_obj;
-    if (PyUnicode_Check(obj)) {
-      bytes_obj = PyUnicode_AsUTF8String(obj);
-      tmp_obj.reset(bytes_obj);
-    } else {
-      bytes_obj = obj;
-    }
-    bytes = PyBytes_AsString(bytes_obj);
-  }
-};
-
-// TODO(wesm): We can just let errors pass through. To be explored later
-#define RETURN_IF_PYERROR()                         \
-  if (PyErr_Occurred()) {                           \
-    PyObject *exc_type, *exc_value, *traceback;     \
-    PyErr_Fetch(&exc_type, &exc_value, &traceback); \
-    PyObjectStringify stringified(exc_value);       \
-    std::string message(stringified.bytes);         \
-    Py_DECREF(exc_type);                            \
-    Py_XDECREF(exc_value);                          \
-    Py_XDECREF(traceback);                          \
-    PyErr_Clear();                                  \
-    return Status::UnknownError(message);           \
-  }
-
-// Return the common PyArrow memory pool
-ARROW_EXPORT void set_default_memory_pool(MemoryPool* pool);
-ARROW_EXPORT MemoryPool* get_memory_pool();
-
-class ARROW_EXPORT NumPyBuffer : public Buffer {
- public:
-  NumPyBuffer(PyArrayObject* arr) : Buffer(nullptr, 0) {
-    arr_ = arr;
-    Py_INCREF(arr);
-
-    data_ = reinterpret_cast<const uint8_t*>(PyArray_DATA(arr_));
-    size_ = PyArray_SIZE(arr_) * PyArray_DESCR(arr_)->elsize;
-    capacity_ = size_;
-  }
-
-  virtual ~NumPyBuffer() { Py_XDECREF(arr_); }
-
- private:
-  PyArrayObject* arr_;
-};
-
-class ARROW_EXPORT PyBuffer : public Buffer {
- public:
-  /// Note that the GIL must be held when calling the PyBuffer constructor.
-  ///
-  /// While memoryview objects support multi-demensional buffers, PyBuffer only supports
-  /// one-dimensional byte buffers.
-  PyBuffer(PyObject* obj);
-  ~PyBuffer();
-
- private:
-  PyObject* obj_;
-};
-
-}  // namespace py
-}  // namespace arrow
-
-#endif  // PYARROW_COMMON_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/config.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/config.cc b/python/src/pyarrow/config.cc
deleted file mode 100644
index 0be6d96..0000000
--- a/python/src/pyarrow/config.cc
+++ /dev/null
@@ -1,35 +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 <Python.h>
-
-#include "pyarrow/config.h"
-
-namespace arrow {
-namespace py {
-
-void pyarrow_init() {}
-
-PyObject* numpy_nan = nullptr;
-
-void pyarrow_set_numpy_nan(PyObject* obj) {
-  Py_INCREF(obj);
-  numpy_nan = obj;
-}
-
-}  // namespace py
-}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/config.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/config.h b/python/src/pyarrow/config.h
deleted file mode 100644
index 87fc5c2..0000000
--- a/python/src/pyarrow/config.h
+++ /dev/null
@@ -1,46 +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.
-
-#ifndef PYARROW_CONFIG_H
-#define PYARROW_CONFIG_H
-
-#include <Python.h>
-
-#include "arrow/util/visibility.h"
-
-#include "pyarrow/numpy_interop.h"
-
-#if PY_MAJOR_VERSION >= 3
-#define PyString_Check PyUnicode_Check
-#endif
-
-namespace arrow {
-namespace py {
-
-ARROW_EXPORT
-extern PyObject* numpy_nan;
-
-ARROW_EXPORT
-void pyarrow_init();
-
-ARROW_EXPORT
-void pyarrow_set_numpy_nan(PyObject* obj);
-
-}  // namespace py
-}  // namespace arrow
-
-#endif  // PYARROW_CONFIG_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/do_import_numpy.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/do_import_numpy.h b/python/src/pyarrow/do_import_numpy.h
deleted file mode 100644
index bb4a382..0000000
--- a/python/src/pyarrow/do_import_numpy.h
+++ /dev/null
@@ -1,21 +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.
-
-// Trick borrowed from dynd-python for initializing the NumPy array API
-
-// Trigger the array import (inversion of NO_IMPORT_ARRAY)
-#define NUMPY_IMPORT_ARRAY

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/helpers.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/helpers.cc b/python/src/pyarrow/helpers.cc
deleted file mode 100644
index 43edf8a..0000000
--- a/python/src/pyarrow/helpers.cc
+++ /dev/null
@@ -1,55 +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 "pyarrow/helpers.h"
-
-#include <arrow/api.h>
-
-namespace arrow {
-namespace py {
-
-#define GET_PRIMITIVE_TYPE(NAME, FACTORY) \
-  case Type::NAME:                        \
-    return FACTORY();                     \
-    break;
-
-std::shared_ptr<DataType> GetPrimitiveType(Type::type type) {
-  switch (type) {
-    case Type::NA:
-      return null();
-      GET_PRIMITIVE_TYPE(UINT8, uint8);
-      GET_PRIMITIVE_TYPE(INT8, int8);
-      GET_PRIMITIVE_TYPE(UINT16, uint16);
-      GET_PRIMITIVE_TYPE(INT16, int16);
-      GET_PRIMITIVE_TYPE(UINT32, uint32);
-      GET_PRIMITIVE_TYPE(INT32, int32);
-      GET_PRIMITIVE_TYPE(UINT64, uint64);
-      GET_PRIMITIVE_TYPE(INT64, int64);
-      GET_PRIMITIVE_TYPE(DATE32, date32);
-      GET_PRIMITIVE_TYPE(DATE64, date64);
-      GET_PRIMITIVE_TYPE(BOOL, boolean);
-      GET_PRIMITIVE_TYPE(FLOAT, float32);
-      GET_PRIMITIVE_TYPE(DOUBLE, float64);
-      GET_PRIMITIVE_TYPE(BINARY, binary);
-      GET_PRIMITIVE_TYPE(STRING, utf8);
-    default:
-      return nullptr;
-  }
-}
-
-}  // namespace py
-}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/3aac4ade/python/src/pyarrow/helpers.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/helpers.h b/python/src/pyarrow/helpers.h
deleted file mode 100644
index 611e814..0000000
--- a/python/src/pyarrow/helpers.h
+++ /dev/null
@@ -1,35 +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.
-
-#ifndef PYARROW_HELPERS_H
-#define PYARROW_HELPERS_H
-
-#include <memory>
-
-#include "arrow/type.h"
-#include "arrow/util/visibility.h"
-
-namespace arrow {
-namespace py {
-
-ARROW_EXPORT
-std::shared_ptr<DataType> GetPrimitiveType(Type::type type);
-
-}  // namespace py
-}  // namespace arrow
-
-#endif  // PYARROW_HELPERS_H