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/13 20:15:59 UTC

[3/3] arrow git commit: ARROW-618: [Python/C++] Support timestamp+timezone conversion to pandas

ARROW-618: [Python/C++] Support timestamp+timezone conversion to pandas

This was a massive pain. This patch brings us up to feature parity with the stuff that was in Feather. The diff is larger than I would like mostly from moving around code in `pyarrow/adapters/pandas.cc`. I suggest we split up that file at our earliest opportunity into the "reader" and "writer" portion at least.

The main work here was refactoring so that the data type for non-object arrays is computed up front (so it might be `timestamp('ns', tz='US/Eastern')`, then we use the visitor pattern to produce the right kind of array. This will also permit implicit type casts and conversions to integer from float because the type metadata is an input parameter.

Things are getting to be a bit of a mess here so we should do some refactoring eventually, and probably also add some microbenchmarks since this stuff is performance sensitive.

I also changed the C++ `pyarrow` namespace to `arrow::py` which will make it less painful to move that code tree to `cpp/src/arrow/python` at some point

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

Closes #375 from wesm/ARROW-618 and squashes the following commits:

4b18bfa [Wes McKinney] Fix rebase conflict
5bc3724 [Wes McKinney] Fix rebase issues
870986f [Wes McKinney] Refactor ArrowSerializer to not be a template and use visitor pattern using passed-in data type. Fix DatetimeTZDtype pandas logic. Arrow Change pyarrow namespace to arrow::py


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

Branch: refs/heads/master
Commit: 00df40ceab48a97fb9f1404ca6a0049e88d0c461
Parents: 331be49
Author: Wes McKinney <we...@twosigma.com>
Authored: Mon Mar 13 16:15:50 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Mon Mar 13 16:15:50 2017 -0400

----------------------------------------------------------------------
 cpp/src/arrow/type.cc                       |   25 +
 cpp/src/arrow/type.h                        |    2 +-
 python/pyarrow/__init__.py                  |    3 +
 python/pyarrow/array.pyx                    |   76 +-
 python/pyarrow/compat.py                    |    9 +
 python/pyarrow/config.pyx                   |    4 +-
 python/pyarrow/feather.py                   |    6 +-
 python/pyarrow/includes/libarrow.pxd        |   11 +-
 python/pyarrow/includes/pyarrow.pxd         |   19 +-
 python/pyarrow/schema.pxd                   |   10 +-
 python/pyarrow/schema.pyx                   |  160 +-
 python/pyarrow/table.pyx                    |   14 +-
 python/pyarrow/tests/test_convert_pandas.py |   33 +-
 python/pyarrow/tests/test_feather.py        |    7 +-
 python/pyarrow/tests/test_schema.py         |  142 +-
 python/src/pyarrow/adapters/builtin.cc      |   61 +-
 python/src/pyarrow/adapters/builtin.h       |   19 +-
 python/src/pyarrow/adapters/pandas-test.cc  |    6 +-
 python/src/pyarrow/adapters/pandas.cc       | 1896 ++++++++++------------
 python/src/pyarrow/adapters/pandas.h        |   44 +-
 python/src/pyarrow/common.cc                |   16 +-
 python/src/pyarrow/common.h                 |   17 +-
 python/src/pyarrow/config.cc                |    6 +-
 python/src/pyarrow/config.h                 |   15 +-
 python/src/pyarrow/helpers.cc               |    8 +-
 python/src/pyarrow/helpers.h                |   15 +-
 python/src/pyarrow/io.cc                    |   12 +-
 python/src/pyarrow/io.h                     |   43 +-
 python/src/pyarrow/numpy_interop.h          |    6 +-
 python/src/pyarrow/type_traits.h            |  212 +++
 python/src/pyarrow/util/datetime.h          |    6 +-
 python/src/pyarrow/util/test_main.cc        |    2 +-
 python/src/pyarrow/visibility.h             |   32 -
 33 files changed, 1595 insertions(+), 1342 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/cpp/src/arrow/type.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.cc b/cpp/src/arrow/type.cc
index 4679a2f..0cafdce 100644
--- a/cpp/src/arrow/type.cc
+++ b/cpp/src/arrow/type.cc
@@ -108,6 +108,31 @@ std::string Date32Type::ToString() const {
   return std::string("date32");
 }
 
+static inline void print_time_unit(TimeUnit unit, std::ostream* stream) {
+  switch (unit) {
+    case TimeUnit::SECOND:
+      (*stream) << "s";
+      break;
+    case TimeUnit::MILLI:
+      (*stream) << "ms";
+      break;
+    case TimeUnit::MICRO:
+      (*stream) << "us";
+      break;
+    case TimeUnit::NANO:
+      (*stream) << "ns";
+      break;
+  }
+}
+
+std::string TimestampType::ToString() const {
+  std::stringstream ss;
+  ss << "timestamp[";
+  print_time_unit(this->unit, &ss);
+  ss << "]";
+  return ss.str();
+}
+
 // ----------------------------------------------------------------------
 // Union type
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/cpp/src/arrow/type.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/type.h b/cpp/src/arrow/type.h
index a838082..15b99c5 100644
--- a/cpp/src/arrow/type.h
+++ b/cpp/src/arrow/type.h
@@ -495,7 +495,7 @@ struct ARROW_EXPORT TimestampType : public FixedWidthType {
   TimestampType(const TimestampType& other) : TimestampType(other.unit) {}
 
   Status Accept(TypeVisitor* visitor) const override;
-  std::string ToString() const override { return name(); }
+  std::string ToString() const override;
   static std::string name() { return "timestamp"; }
 
   TimeUnit unit;

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/__init__.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/__init__.py b/python/pyarrow/__init__.py
index 6724b52..a4aac44 100644
--- a/python/pyarrow/__init__.py
+++ b/python/pyarrow/__init__.py
@@ -56,6 +56,8 @@ from pyarrow.scalar import (ArrayValue, Scalar, NA, NAType,
                             FloatValue, DoubleValue, ListValue,
                             BinaryValue, StringValue)
 
+import pyarrow.schema as _schema
+
 from pyarrow.schema import (null, bool_,
                             int8, int16, int32, int64,
                             uint8, uint16, uint32, uint64,
@@ -64,6 +66,7 @@ from pyarrow.schema import (null, bool_,
                             list_, struct, dictionary, field,
                             DataType, Field, Schema, schema)
 
+
 from pyarrow.table import Column, RecordBatch, Table, concat_tables
 
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/array.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/array.pyx b/python/pyarrow/array.pyx
index 6a6b4ba..11244e7 100644
--- a/python/pyarrow/array.pyx
+++ b/python/pyarrow/array.pyx
@@ -34,7 +34,8 @@ from pyarrow.memory cimport MemoryPool, maybe_unbox_memory_pool
 cimport pyarrow.scalar as scalar
 from pyarrow.scalar import NA
 
-from pyarrow.schema cimport Field, Schema, DictionaryType
+from pyarrow.schema cimport (DataType, Field, Schema, DictionaryType,
+                             box_data_type)
 import pyarrow.schema as schema
 
 cimport cpython
@@ -45,16 +46,40 @@ cdef _pandas():
     return pd
 
 
+cdef maybe_coerce_datetime64(values, dtype, DataType type,
+                             timestamps_to_ms=False):
+
+    from pyarrow.compat import DatetimeTZDtype
+
+    if values.dtype.type != np.datetime64:
+        return values, type
+
+    coerce_ms = timestamps_to_ms and values.dtype != 'datetime64[ms]'
+
+    if coerce_ms:
+        values = values.astype('datetime64[ms]')
+
+    if isinstance(dtype, DatetimeTZDtype):
+        tz = dtype.tz
+        unit = 'ms' if coerce_ms else dtype.unit
+        type = schema.timestamp(unit, tz)
+    else:
+        # Trust the NumPy dtype
+        type = schema.type_from_numpy_dtype(values.dtype)
+
+    return values, type
+
+
 cdef class Array:
 
     cdef init(self, const shared_ptr[CArray]& sp_array):
         self.sp_array = sp_array
         self.ap = sp_array.get()
-        self.type = DataType()
-        self.type.init(self.sp_array.get().type())
+        self.type = box_data_type(self.sp_array.get().type())
 
     @staticmethod
-    def from_pandas(obj, mask=None, timestamps_to_ms=False, Field field=None,
+    def from_pandas(obj, mask=None, DataType type=None,
+                    timestamps_to_ms=False,
                     MemoryPool memory_pool=None):
         """
         Convert pandas.Series to an Arrow Array.
@@ -66,6 +91,9 @@ cdef class Array:
         mask : pandas.Series or numpy.ndarray, optional
             boolean mask if the object is valid or null
 
+        type : pyarrow.DataType
+            Explicit type to attempt to coerce to
+
         timestamps_to_ms : bool, optional
             Convert datetime columns to ms resolution. This is needed for
             compatibility with other functionality like Parquet I/O which
@@ -107,33 +135,43 @@ cdef class Array:
         """
         cdef:
             shared_ptr[CArray] out
-            shared_ptr[CField] c_field
+            shared_ptr[CDataType] c_type
             CMemoryPool* pool
 
         pd = _pandas()
 
-        if field is not None:
-            c_field = field.sp_field
-
         if mask is not None:
             mask = get_series_values(mask)
 
-        series_values = get_series_values(obj)
+        values = get_series_values(obj)
+        pool = maybe_unbox_memory_pool(memory_pool)
 
-        if isinstance(series_values, pd.Categorical):
+        if isinstance(values, pd.Categorical):
             return DictionaryArray.from_arrays(
-                series_values.codes, series_values.categories.values,
+                values.codes, values.categories.values,
                 mask=mask, memory_pool=memory_pool)
+        elif values.dtype == object:
+            # Object dtype undergoes a different conversion path as more type
+            # inference may be needed
+            if type is not None:
+                c_type = type.sp_type
+            with nogil:
+                check_status(pyarrow.PandasObjectsToArrow(
+                    pool, values, mask, c_type, &out))
         else:
-            if series_values.dtype.type == np.datetime64 and timestamps_to_ms:
-                series_values = series_values.astype('datetime64[ms]')
+            values, type = maybe_coerce_datetime64(
+                values, obj.dtype, type, timestamps_to_ms=timestamps_to_ms)
+
+            if type is None:
+                check_status(pyarrow.PandasDtypeToArrow(values.dtype, &c_type))
+            else:
+                c_type = type.sp_type
 
-            pool = maybe_unbox_memory_pool(memory_pool)
             with nogil:
                 check_status(pyarrow.PandasToArrow(
-                    pool, series_values, mask, c_field, &out))
+                    pool, values, mask, c_type, &out))
 
-            return box_array(out)
+        return box_array(out)
 
     @staticmethod
     def from_list(object list_obj, DataType type=None,
@@ -338,6 +376,10 @@ cdef class DateArray(NumericArray):
     pass
 
 
+cdef class TimestampArray(NumericArray):
+    pass
+
+
 cdef class FloatArray(FloatingPointArray):
     pass
 
@@ -423,7 +465,7 @@ cdef dict _array_classes = {
     Type_LIST: ListArray,
     Type_BINARY: BinaryArray,
     Type_STRING: StringArray,
-    Type_TIMESTAMP: Int64Array,
+    Type_TIMESTAMP: TimestampArray,
     Type_DICTIONARY: DictionaryArray
 }
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/compat.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/compat.py b/python/pyarrow/compat.py
index 9148be7..74d7ca2 100644
--- a/python/pyarrow/compat.py
+++ b/python/pyarrow/compat.py
@@ -17,9 +17,11 @@
 
 # flake8: noqa
 
+from distutils.version import LooseVersion
 import itertools
 
 import numpy as np
+import pandas as pd
 
 import sys
 import six
@@ -115,6 +117,13 @@ def encode_file_path(path):
     return encoded_path
 
 
+if LooseVersion(pd.__version__) < '0.19.0':
+    pdapi = pd.core.common
+    from pandas.core.dtypes import DatetimeTZDtype
+else:
+    from pandas.types.dtypes import DatetimeTZDtype
+    pdapi = pd.api.types
+
 integer_types = six.integer_types + (np.integer,)
 
 __all__ = []

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/config.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/config.pyx b/python/pyarrow/config.pyx
index aa30f09..5ad7cf5 100644
--- a/python/pyarrow/config.pyx
+++ b/python/pyarrow/config.pyx
@@ -17,10 +17,10 @@
 cdef extern from 'pyarrow/do_import_numpy.h':
     pass
 
-cdef extern from 'pyarrow/numpy_interop.h' namespace 'pyarrow':
+cdef extern from 'pyarrow/numpy_interop.h' namespace 'arrow::py':
     int import_numpy()
 
-cdef extern from 'pyarrow/config.h' namespace 'pyarrow':
+cdef extern from 'pyarrow/config.h' namespace 'arrow::py':
     void pyarrow_init()
     void pyarrow_set_numpy_nan(object o)
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/feather.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/feather.py b/python/pyarrow/feather.py
index b7dbf96..28424af 100644
--- a/python/pyarrow/feather.py
+++ b/python/pyarrow/feather.py
@@ -19,6 +19,7 @@ import six
 from distutils.version import LooseVersion
 import pandas as pd
 
+from pyarrow.compat import pdapi
 from pyarrow._feather import FeatherError  # noqa
 from pyarrow.table import Table
 import pyarrow._feather as ext
@@ -27,11 +28,6 @@ import pyarrow._feather as ext
 if LooseVersion(pd.__version__) < '0.17.0':
     raise ImportError("feather requires pandas >= 0.17.0")
 
-if LooseVersion(pd.__version__) < '0.19.0':
-    pdapi = pd.core.common
-else:
-    pdapi = pd.api.types
-
 
 class FeatherReader(ext.FeatherReader):
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/includes/libarrow.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/libarrow.pxd b/python/pyarrow/includes/libarrow.pxd
index 253cabb..dee7fd4 100644
--- a/python/pyarrow/includes/libarrow.pxd
+++ b/python/pyarrow/includes/libarrow.pxd
@@ -84,6 +84,13 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         shared_ptr[CArray] indices()
         shared_ptr[CArray] dictionary()
 
+    cdef cppclass CTimestampType" arrow::TimestampType"(CFixedWidthType):
+        TimeUnit unit
+        c_string timezone
+
+    cdef cppclass CTimeType" arrow::TimeType"(CFixedWidthType):
+        TimeUnit unit
+
     cdef cppclass CDictionaryType" arrow::DictionaryType"(CFixedWidthType):
         CDictionaryType(const shared_ptr[CDataType]& index_type,
                         const shared_ptr[CArray]& dictionary)
@@ -92,6 +99,7 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
         shared_ptr[CArray] dictionary()
 
     shared_ptr[CDataType] timestamp(TimeUnit unit)
+    shared_ptr[CDataType] timestamp(const c_string& timezone, TimeUnit unit)
 
     cdef cppclass CMemoryPool" arrow::MemoryPool":
         int64_t bytes_allocated()
@@ -117,9 +125,6 @@ cdef extern from "arrow/api.h" namespace "arrow" nogil:
     cdef cppclass CStringType" arrow::StringType"(CDataType):
         pass
 
-    cdef cppclass CTimestampType" arrow::TimestampType"(CDataType):
-        TimeUnit unit
-
     cdef cppclass CField" arrow::Field":
         c_string name
         shared_ptr[CDataType] type

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/includes/pyarrow.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/pyarrow.pxd b/python/pyarrow/includes/pyarrow.pxd
index f1d45e0..9fbddba 100644
--- a/python/pyarrow/includes/pyarrow.pxd
+++ b/python/pyarrow/includes/pyarrow.pxd
@@ -18,22 +18,29 @@
 # distutils: language = c++
 
 from pyarrow.includes.common cimport *
-from pyarrow.includes.libarrow cimport (CArray, CBuffer, CColumn, CField,
+from pyarrow.includes.libarrow cimport (CArray, CBuffer, CColumn,
                                         CTable, CDataType, CStatus, Type,
                                         CMemoryPool, TimeUnit)
 
 cimport pyarrow.includes.libarrow_io as arrow_io
 
 
-cdef extern from "pyarrow/api.h" namespace "pyarrow" nogil:
+cdef extern from "pyarrow/api.h" namespace "arrow::py" nogil:
     shared_ptr[CDataType] GetPrimitiveType(Type type)
     shared_ptr[CDataType] GetTimestampType(TimeUnit unit)
-    CStatus ConvertPySequence(object obj, CMemoryPool* pool, shared_ptr[CArray]* out)
+    CStatus ConvertPySequence(object obj, CMemoryPool* pool,
+                              shared_ptr[CArray]* out)
+
+    CStatus PandasDtypeToArrow(object dtype, shared_ptr[CDataType]* type)
 
     CStatus PandasToArrow(CMemoryPool* pool, object ao, object mo,
-                          shared_ptr[CField] field,
+                          const shared_ptr[CDataType]& type,
                           shared_ptr[CArray]* out)
 
+    CStatus PandasObjectsToArrow(CMemoryPool* pool, object ao, object mo,
+                                 const shared_ptr[CDataType]& type,
+                                 shared_ptr[CArray]* out)
+
     CStatus ConvertArrayToPandas(const shared_ptr[CArray]& arr,
                                  PyObject* py_ref, PyObject** out)
 
@@ -47,12 +54,12 @@ cdef extern from "pyarrow/api.h" namespace "pyarrow" nogil:
     CMemoryPool* get_memory_pool()
 
 
-cdef extern from "pyarrow/common.h" namespace "pyarrow" nogil:
+cdef extern from "pyarrow/common.h" namespace "arrow::py" nogil:
     cdef cppclass PyBytesBuffer(CBuffer):
         PyBytesBuffer(object o)
 
 
-cdef extern from "pyarrow/io.h" namespace "pyarrow" nogil:
+cdef extern from "pyarrow/io.h" namespace "arrow::py" nogil:
     cdef cppclass PyReadableFile(arrow_io.ReadableFileInterface):
         PyReadableFile(object fo)
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/schema.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/schema.pxd b/python/pyarrow/schema.pxd
index 390954c..15ee5f1 100644
--- a/python/pyarrow/schema.pxd
+++ b/python/pyarrow/schema.pxd
@@ -16,7 +16,9 @@
 # under the License.
 
 from pyarrow.includes.common cimport *
-from pyarrow.includes.libarrow cimport (CDataType, CDictionaryType,
+from pyarrow.includes.libarrow cimport (CDataType,
+                                        CDictionaryType,
+                                        CTimestampType,
                                         CField, CSchema)
 
 cdef class DataType:
@@ -31,6 +33,12 @@ cdef class DictionaryType(DataType):
     cdef:
         const CDictionaryType* dict_type
 
+
+cdef class TimestampType(DataType):
+    cdef:
+        const CTimestampType* ts_type
+
+
 cdef class Field:
     cdef:
         shared_ptr[CField] sp_field

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/schema.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/schema.pyx b/python/pyarrow/schema.pyx
index d636b5a..4bc938d 100644
--- a/python/pyarrow/schema.pyx
+++ b/python/pyarrow/schema.pyx
@@ -26,23 +26,19 @@ from cython.operator cimport dereference as deref
 
 from pyarrow.compat import frombytes, tobytes
 from pyarrow.array cimport Array
+from pyarrow.error cimport check_status
 from pyarrow.includes.libarrow cimport (CDataType, CStructType, CListType,
-                                        Type_NA, Type_BOOL,
-                                        Type_UINT8, Type_INT8,
-                                        Type_UINT16, Type_INT16,
-                                        Type_UINT32, Type_INT32,
-                                        Type_UINT64, Type_INT64,
-                                        Type_TIMESTAMP, Type_DATE,
-                                        Type_FLOAT, Type_DOUBLE,
-                                        Type_STRING, Type_BINARY,
                                         TimeUnit_SECOND, TimeUnit_MILLI,
                                         TimeUnit_MICRO, TimeUnit_NANO,
                                         Type, TimeUnit)
 cimport pyarrow.includes.pyarrow as pyarrow
-cimport pyarrow.includes.libarrow as libarrow
+cimport pyarrow.includes.libarrow as la
 
 cimport cpython
 
+import six
+
+
 cdef class DataType:
 
     def __cinit__(self):
@@ -73,13 +69,33 @@ cdef class DictionaryType(DataType):
         DataType.init(self, type)
         self.dict_type = <const CDictionaryType*> type.get()
 
-    def __str__(self):
-        return frombytes(self.type.ToString())
-
     def __repr__(self):
         return 'DictionaryType({0})'.format(str(self))
 
 
+cdef class TimestampType(DataType):
+
+    cdef init(self, const shared_ptr[CDataType]& type):
+        DataType.init(self, type)
+        self.ts_type = <const CTimestampType*> type.get()
+
+    property unit:
+
+        def __get__(self):
+            return timeunit_to_string(self.ts_type.unit)
+
+    property tz:
+
+        def __get__(self):
+            if self.ts_type.timezone.size() > 0:
+                return frombytes(self.ts_type.timezone)
+            else:
+                return None
+
+    def __repr__(self):
+        return 'TimestampType({0})'.format(str(self))
+
+
 cdef class Field:
 
     def __cinit__(self):
@@ -205,49 +221,76 @@ cdef DataType primitive_type(Type type):
 def field(name, type, bint nullable=True):
     return Field.from_py(name, type, nullable)
 
+
 cdef set PRIMITIVE_TYPES = set([
-    Type_NA, Type_BOOL,
-    Type_UINT8, Type_INT8,
-    Type_UINT16, Type_INT16,
-    Type_UINT32, Type_INT32,
-    Type_UINT64, Type_INT64,
-    Type_TIMESTAMP, Type_DATE,
-    Type_FLOAT, Type_DOUBLE])
+    la.Type_NA, la.Type_BOOL,
+    la.Type_UINT8, la.Type_INT8,
+    la.Type_UINT16, la.Type_INT16,
+    la.Type_UINT32, la.Type_INT32,
+    la.Type_UINT64, la.Type_INT64,
+    la.Type_TIMESTAMP, la.Type_DATE,
+    la.Type_FLOAT, la.Type_DOUBLE])
+
 
 def null():
-    return primitive_type(Type_NA)
+    return primitive_type(la.Type_NA)
+
 
 def bool_():
-    return primitive_type(Type_BOOL)
+    return primitive_type(la.Type_BOOL)
+
 
 def uint8():
-    return primitive_type(Type_UINT8)
+    return primitive_type(la.Type_UINT8)
+
 
 def int8():
-    return primitive_type(Type_INT8)
+    return primitive_type(la.Type_INT8)
+
 
 def uint16():
-    return primitive_type(Type_UINT16)
+    return primitive_type(la.Type_UINT16)
+
 
 def int16():
-    return primitive_type(Type_INT16)
+    return primitive_type(la.Type_INT16)
+
 
 def uint32():
-    return primitive_type(Type_UINT32)
+    return primitive_type(la.Type_UINT32)
+
 
 def int32():
-    return primitive_type(Type_INT32)
+    return primitive_type(la.Type_INT32)
+
 
 def uint64():
-    return primitive_type(Type_UINT64)
+    return primitive_type(la.Type_UINT64)
+
 
 def int64():
-    return primitive_type(Type_INT64)
+    return primitive_type(la.Type_INT64)
+
 
 cdef dict _timestamp_type_cache = {}
 
-def timestamp(unit_str):
-    cdef TimeUnit unit
+
+cdef timeunit_to_string(TimeUnit unit):
+    if unit == TimeUnit_SECOND:
+        return 's'
+    elif unit == TimeUnit_MILLI:
+        return 'ms'
+    elif unit == TimeUnit_MICRO:
+        return 'us'
+    elif unit == TimeUnit_NANO:
+        return 'ns'
+
+
+def timestamp(unit_str, tz=None):
+    cdef:
+        TimeUnit unit
+        c_string c_timezone
+
     if unit_str == "s":
         unit = TimeUnit_SECOND
     elif unit_str == 'ms':
@@ -259,34 +302,47 @@ def timestamp(unit_str):
     else:
         raise TypeError('Invalid TimeUnit string')
 
-    if unit in _timestamp_type_cache:
-        return _timestamp_type_cache[unit]
+    cdef TimestampType out = TimestampType()
+
+    if tz is None:
+        out.init(la.timestamp(unit))
+        if unit in _timestamp_type_cache:
+            return _timestamp_type_cache[unit]
+        _timestamp_type_cache[unit] = out
+    else:
+        if not isinstance(tz, six.string_types):
+            tz = tz.zone
+
+        c_timezone = tobytes(tz)
+        out.init(la.timestamp(c_timezone, unit))
 
-    cdef DataType out = DataType()
-    out.init(libarrow.timestamp(unit))
-    _timestamp_type_cache[unit] = out
     return out
 
+
 def date():
-    return primitive_type(Type_DATE)
+    return primitive_type(la.Type_DATE)
+
 
 def float_():
-    return primitive_type(Type_FLOAT)
+    return primitive_type(la.Type_FLOAT)
+
 
 def double():
-    return primitive_type(Type_DOUBLE)
+    return primitive_type(la.Type_DOUBLE)
+
 
 def string():
     """
     UTF8 string
     """
-    return primitive_type(Type_STRING)
+    return primitive_type(la.Type_STRING)
+
 
 def binary():
     """
     Binary (PyBytes-like) type
     """
-    return primitive_type(Type_BINARY)
+    return primitive_type(la.Type_BINARY)
 
 
 def list_(DataType value_type):
@@ -326,13 +382,25 @@ def struct(fields):
     out.init(struct_type)
     return out
 
+
 def schema(fields):
     return Schema.from_fields(fields)
 
+
 cdef DataType box_data_type(const shared_ptr[CDataType]& type):
+    cdef:
+        DataType out
+
     if type.get() == NULL:
         return None
-    cdef DataType out = DataType()
+
+    if type.get().type == la.Type_DICTIONARY:
+        out = DictionaryType()
+    elif type.get().type == la.Type_TIMESTAMP:
+        out = TimestampType()
+    else:
+        out = DataType()
+
     out.init(type)
     return out
 
@@ -347,3 +415,11 @@ cdef Schema box_schema(const shared_ptr[CSchema]& type):
     cdef Schema out = Schema()
     out.init_schema(type)
     return out
+
+
+def type_from_numpy_dtype(object dtype):
+    cdef shared_ptr[CDataType] c_type
+    with nogil:
+        check_status(pyarrow.PandasDtypeToArrow(dtype, &c_type))
+
+    return box_data_type(c_type)

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/table.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/table.pyx b/python/pyarrow/table.pyx
index 5657b97..58f5d68 100644
--- a/python/pyarrow/table.pyx
+++ b/python/pyarrow/table.pyx
@@ -30,7 +30,7 @@ import pyarrow.config
 from pyarrow.array cimport Array, box_array, wrap_array_output
 from pyarrow.error import ArrowException
 from pyarrow.error cimport check_status
-from pyarrow.schema cimport box_data_type, box_schema, Field
+from pyarrow.schema cimport box_data_type, box_schema, DataType
 
 from pyarrow.compat import frombytes, tobytes
 
@@ -302,14 +302,15 @@ cdef _dataframe_to_arrays(df, name, timestamps_to_ms, Schema schema):
     cdef:
         list names = []
         list arrays = []
-        Field field = None
+        DataType type = None
 
     for name in df.columns:
         col = df[name]
         if schema is not None:
-            field = schema.field_by_name(name)
-        arr = Array.from_pandas(col, timestamps_to_ms=timestamps_to_ms,
-                                field=field)
+            type = schema.field_by_name(name).type
+
+        arr = Array.from_pandas(col, type=type,
+                                timestamps_to_ms=timestamps_to_ms)
         names.append(name)
         arrays.append(arr)
 
@@ -522,6 +523,7 @@ cdef table_to_blockmanager(const shared_ptr[CTable]& table, int nthreads):
 
     import pandas.core.internals as _int
     from pandas import RangeIndex, Categorical
+    from pyarrow.compat import DatetimeTZDtype
 
     with nogil:
         check_status(pyarrow.ConvertTableToPandas(table, nthreads,
@@ -541,9 +543,9 @@ cdef table_to_blockmanager(const shared_ptr[CTable]& table, int nthreads):
                                     klass=_int.CategoricalBlock,
                                     fastpath=True)
         elif 'timezone' in item:
-            from pandas.types.api import DatetimeTZDtype
             dtype = DatetimeTZDtype('ns', tz=item['timezone'])
             block = _int.make_block(block_arr, placement=placement,
+                                    klass=_int.DatetimeTZBlock,
                                     dtype=dtype, fastpath=True)
         else:
             block = _int.make_block(block_arr, placement=placement)

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/tests/test_convert_pandas.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_convert_pandas.py b/python/pyarrow/tests/test_convert_pandas.py
index a79bb23..6b89444 100644
--- a/python/pyarrow/tests/test_convert_pandas.py
+++ b/python/pyarrow/tests/test_convert_pandas.py
@@ -77,9 +77,9 @@ class TestPandasConversion(unittest.TestCase):
         tm.assert_frame_equal(result, expected, check_dtype=check_dtype)
 
     def _check_array_roundtrip(self, values, expected=None,
-                               timestamps_to_ms=False, field=None):
+                               timestamps_to_ms=False, type=None):
         arr = A.Array.from_pandas(values, timestamps_to_ms=timestamps_to_ms,
-                                  field=field)
+                                  type=type)
         result = arr.to_pandas()
 
         assert arr.null_count == pd.isnull(values).sum()
@@ -134,11 +134,13 @@ class TestPandasConversion(unittest.TestCase):
         data = OrderedDict()
         fields = []
 
-        numpy_dtypes = [('i1', A.int8()), ('i2', A.int16()),
-                        ('i4', A.int32()), ('i8', A.int64()),
-                        ('u1', A.uint8()), ('u2', A.uint16()),
-                        ('u4', A.uint32()), ('u8', A.uint64()),
-                        ('longlong', A.int64()), ('ulonglong', A.uint64())]
+        numpy_dtypes = [
+            ('i1', A.int8()), ('i2', A.int16()),
+            ('i4', A.int32()), ('i8', A.int64()),
+            ('u1', A.uint8()), ('u2', A.uint16()),
+            ('u4', A.uint32()), ('u8', A.uint64()),
+            ('longlong', A.int64()), ('ulonglong', A.uint64())
+        ]
         num_values = 100
 
         for dtype, arrow_dtype in numpy_dtypes:
@@ -153,7 +155,6 @@ class TestPandasConversion(unittest.TestCase):
         schema = A.Schema.from_fields(fields)
         self._check_pandas_roundtrip(df, expected_schema=schema)
 
-
     def test_integer_with_nulls(self):
         # pandas requires upcast to float dtype
 
@@ -301,9 +302,9 @@ class TestPandasConversion(unittest.TestCase):
                 '2010-08-13T05:46:57.437'],
                 dtype='datetime64[ms]')
             })
-        df_est = df['datetime64'].dt.tz_localize('US/Eastern').to_frame()
-        df_utc = df_est['datetime64'].dt.tz_convert('UTC').to_frame()
-        self._check_pandas_roundtrip(df_est, expected=df_utc, timestamps_to_ms=True, check_dtype=False)
+        df['datetime64'] = (df['datetime64'].dt.tz_localize('US/Eastern')
+                            .to_frame())
+        self._check_pandas_roundtrip(df, timestamps_to_ms=True)
 
         # drop-in a null and ns instead of ms
         df = pd.DataFrame({
@@ -314,9 +315,9 @@ class TestPandasConversion(unittest.TestCase):
                 '2010-08-13T05:46:57.437699912'],
                 dtype='datetime64[ns]')
             })
-        df_est = df['datetime64'].dt.tz_localize('US/Eastern').to_frame()
-        df_utc = df_est['datetime64'].dt.tz_convert('UTC').to_frame()
-        self._check_pandas_roundtrip(df_est, expected=df_utc, timestamps_to_ms=False, check_dtype=False)
+        df['datetime64'] = (df['datetime64'].dt.tz_localize('US/Eastern')
+                            .to_frame())
+        self._check_pandas_roundtrip(df, timestamps_to_ms=False)
 
     def test_date(self):
         df = pd.DataFrame({
@@ -341,7 +342,7 @@ class TestPandasConversion(unittest.TestCase):
 
         for column in df.columns:
             field = schema.field_by_name(column)
-            self._check_array_roundtrip(df[column], field=field)
+            self._check_array_roundtrip(df[column], type=field.type)
 
     def test_column_of_lists(self):
         df, schema = dataframe_with_lists()
@@ -351,7 +352,7 @@ class TestPandasConversion(unittest.TestCase):
 
         for column in df.columns:
             field = schema.field_by_name(column)
-            self._check_array_roundtrip(df[column], field=field)
+            self._check_array_roundtrip(df[column], type=field.type)
 
     def test_threaded_conversion(self):
         df = _alltypes_example()

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/tests/test_feather.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_feather.py b/python/pyarrow/tests/test_feather.py
index 451475b..e4b6273 100644
--- a/python/pyarrow/tests/test_feather.py
+++ b/python/pyarrow/tests/test_feather.py
@@ -23,8 +23,8 @@ import numpy as np
 from pandas.util.testing import assert_frame_equal
 import pandas as pd
 
+import pyarrow as pa
 from pyarrow.compat import guid
-from pyarrow.error import ArrowException
 from pyarrow.feather import (read_feather, write_feather,
                              FeatherReader)
 from pyarrow._feather import FeatherWriter
@@ -47,7 +47,7 @@ class TestFeatherReader(unittest.TestCase):
                 pass
 
     def test_file_not_exist(self):
-        with self.assertRaises(ArrowException):
+        with self.assertRaises(pa.ArrowException):
             FeatherReader('test_invalid_file')
 
     def _get_null_counts(self, path, columns=None):
@@ -291,7 +291,6 @@ class TestFeatherReader(unittest.TestCase):
         self._check_pandas_roundtrip(df, expected,
                                      null_counts=[2 * repeats])
 
-    @pytest.mark.xfail
     def test_timestamp(self):
         df = pd.DataFrame({'naive': pd.date_range('2016-03-28', periods=10)})
         df['with_tz'] = (df.naive.dt.tz_localize('utc')
@@ -299,7 +298,6 @@ class TestFeatherReader(unittest.TestCase):
 
         self._check_pandas_roundtrip(df)
 
-    @pytest.mark.xfail
     def test_timestamp_with_nulls(self):
         df = pd.DataFrame({'test': [pd.datetime(2016, 1, 1),
                                     None,
@@ -308,7 +306,6 @@ class TestFeatherReader(unittest.TestCase):
 
         self._check_pandas_roundtrip(df, null_counts=[1, 1])
 
-    @pytest.mark.xfail
     def test_out_of_float64_timestamp_with_nulls(self):
         df = pd.DataFrame(
             {'test': pd.DatetimeIndex([1451606400000000001,

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/pyarrow/tests/test_schema.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_schema.py b/python/pyarrow/tests/test_schema.py
index dd68f39..5588840 100644
--- a/python/pyarrow/tests/test_schema.py
+++ b/python/pyarrow/tests/test_schema.py
@@ -15,82 +15,108 @@
 # specific language governing permissions and limitations
 # under the License.
 
-from pyarrow.compat import unittest
-import pyarrow as arrow
+import pytest
 
-A = arrow
+import pyarrow as pa
 
+import numpy as np
 
-class TestTypes(unittest.TestCase):
+# XXX: pyarrow.schema.schema masks the module on imports
+sch = pa._schema
 
-    def test_integers(self):
-        dtypes = ['int8', 'int16', 'int32', 'int64',
-                  'uint8', 'uint16', 'uint32', 'uint64']
 
-        for name in dtypes:
-            factory = getattr(arrow, name)
-            t = factory()
-            assert str(t) == name
+def test_type_integers():
+    dtypes = ['int8', 'int16', 'int32', 'int64',
+              'uint8', 'uint16', 'uint32', 'uint64']
 
-    def test_list(self):
-        value_type = arrow.int32()
-        list_type = arrow.list_(value_type)
-        assert str(list_type) == 'list<item: int32>'
+    for name in dtypes:
+        factory = getattr(pa, name)
+        t = factory()
+        assert str(t) == name
 
-    def test_string(self):
-        t = arrow.string()
-        assert str(t) == 'string'
 
-    def test_field(self):
-        t = arrow.string()
-        f = arrow.field('foo', t)
+def test_type_list():
+    value_type = pa.int32()
+    list_type = pa.list_(value_type)
+    assert str(list_type) == 'list<item: int32>'
 
-        assert f.name == 'foo'
-        assert f.nullable
-        assert f.type is t
-        assert repr(f) == "Field('foo', type=string)"
 
-        f = arrow.field('foo', t, False)
-        assert not f.nullable
+def test_type_string():
+    t = pa.string()
+    assert str(t) == 'string'
 
-    def test_schema(self):
-        fields = [
-            A.field('foo', A.int32()),
-            A.field('bar', A.string()),
-            A.field('baz', A.list_(A.int8()))
-        ]
-        sch = A.schema(fields)
 
-        assert len(sch) == 3
-        assert sch[0].name == 'foo'
-        assert sch[0].type == fields[0].type
-        assert sch.field_by_name('foo').name == 'foo'
-        assert sch.field_by_name('foo').type == fields[0].type
+def test_type_timestamp_with_tz():
+    tz = 'America/Los_Angeles'
+    t = pa.timestamp('ns', tz=tz)
+    assert t.unit == 'ns'
+    assert t.tz == tz
 
-        assert repr(sch) == """\
+
+def test_type_from_numpy_dtype_timestamps():
+    cases = [
+        (np.dtype('datetime64[s]'), pa.timestamp('s')),
+        (np.dtype('datetime64[ms]'), pa.timestamp('ms')),
+        (np.dtype('datetime64[us]'), pa.timestamp('us')),
+        (np.dtype('datetime64[ns]'), pa.timestamp('ns'))
+    ]
+
+    for dt, pt in cases:
+        result = sch.type_from_numpy_dtype(dt)
+        assert result == pt
+
+
+def test_field():
+    t = pa.string()
+    f = pa.field('foo', t)
+
+    assert f.name == 'foo'
+    assert f.nullable
+    assert f.type is t
+    assert repr(f) == "Field('foo', type=string)"
+
+    f = pa.field('foo', t, False)
+    assert not f.nullable
+
+
+def test_schema():
+    fields = [
+        pa.field('foo', pa.int32()),
+        pa.field('bar', pa.string()),
+        pa.field('baz', pa.list_(pa.int8()))
+    ]
+    sch = pa.schema(fields)
+
+    assert len(sch) == 3
+    assert sch[0].name == 'foo'
+    assert sch[0].type == fields[0].type
+    assert sch.field_by_name('foo').name == 'foo'
+    assert sch.field_by_name('foo').type == fields[0].type
+
+    assert repr(sch) == """\
 foo: int32
 bar: string
 baz: list<item: int8>"""
 
-    def test_schema_equals(self):
-        fields = [
-            A.field('foo', A.int32()),
-            A.field('bar', A.string()),
-            A.field('baz', A.list_(A.int8()))
-        ]
 
-        sch1 = A.schema(fields)
-        print(dir(sch1))
-        sch2 = A.schema(fields)
-        assert sch1.equals(sch2)
+def test_field_empty():
+    f = pa.Field()
+    with pytest.raises(ReferenceError):
+        repr(f)
+
 
-        del fields[-1]
-        sch3 = A.schema(fields)
-        assert not sch1.equals(sch3)
+def test_schema_equals():
+    fields = [
+        pa.field('foo', pa.int32()),
+        pa.field('bar', pa.string()),
+        pa.field('baz', pa.list_(pa.int8()))
+    ]
 
+    sch1 = pa.schema(fields)
+    print(dir(sch1))
+    sch2 = pa.schema(fields)
+    assert sch1.equals(sch2)
 
-class TestField(unittest.TestCase):
-    def test_empty_field(self):
-        f = arrow.Field()
-        with self.assertRaises(ReferenceError):
-            repr(f)
+    del fields[-1]
+    sch3 = pa.schema(fields)
+    assert not sch1.equals(sch3)

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/src/pyarrow/adapters/builtin.cc
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/builtin.cc b/python/src/pyarrow/adapters/builtin.cc
index 4f7b2cb..b197f58 100644
--- a/python/src/pyarrow/adapters/builtin.cc
+++ b/python/src/pyarrow/adapters/builtin.cc
@@ -27,13 +27,8 @@
 #include "pyarrow/helpers.h"
 #include "pyarrow/util/datetime.h"
 
-using arrow::ArrayBuilder;
-using arrow::DataType;
-using arrow::MemoryPool;
-using arrow::Status;
-using arrow::Type;
-
-namespace pyarrow {
+namespace arrow {
+namespace py {
 
 static inline bool IsPyInteger(PyObject* obj) {
 #if PYARROW_IS_PY2
@@ -82,22 +77,22 @@ class ScalarVisitor {
   std::shared_ptr<DataType> GetType() {
     // TODO(wesm): handling mixed-type cases
     if (float_count_) {
-      return arrow::float64();
+      return float64();
     } else if (int_count_) {
       // TODO(wesm): tighter type later
-      return arrow::int64();
+      return int64();
     } else if (date_count_) {
-      return arrow::date();
+      return date();
     } else if (timestamp_count_) {
-      return arrow::timestamp(arrow::TimeUnit::MICRO);
+      return timestamp(TimeUnit::MICRO);
     } else if (bool_count_) {
-      return arrow::boolean();
+      return boolean();
     } else if (binary_count_) {
-      return arrow::binary();
+      return binary();
     } else if (unicode_count_) {
-      return arrow::utf8();
+      return utf8();
     } else {
-      return arrow::null();
+      return null();
     }
   }
 
@@ -157,14 +152,14 @@ class SeqVisitor {
   std::shared_ptr<DataType> GetType() {
     if (scalars_.total_count() == 0) {
       if (max_nesting_level_ == 0) {
-        return arrow::null();
+        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<arrow::ListType>(result);
+        result = std::make_shared<ListType>(result);
       }
       return result;
     }
@@ -215,7 +210,7 @@ Status InferArrowType(PyObject* obj, int64_t* size, std::shared_ptr<DataType>* o
   }
 
   // For 0-length sequences, refuse to guess
-  if (*size == 0) { *out_type = arrow::null(); }
+  if (*size == 0) { *out_type = null(); }
 
   SeqVisitor seq_visitor;
   RETURN_NOT_OK(seq_visitor.Visit(obj));
@@ -255,7 +250,7 @@ class TypedConverter : public SeqConverter {
   BuilderType* typed_builder_;
 };
 
-class BoolConverter : public TypedConverter<arrow::BooleanBuilder> {
+class BoolConverter : public TypedConverter<BooleanBuilder> {
  public:
   Status AppendData(PyObject* seq) override {
     Py_ssize_t size = PySequence_Size(seq);
@@ -276,7 +271,7 @@ class BoolConverter : public TypedConverter<arrow::BooleanBuilder> {
   }
 };
 
-class Int64Converter : public TypedConverter<arrow::Int64Builder> {
+class Int64Converter : public TypedConverter<Int64Builder> {
  public:
   Status AppendData(PyObject* seq) override {
     int64_t val;
@@ -296,7 +291,7 @@ class Int64Converter : public TypedConverter<arrow::Int64Builder> {
   }
 };
 
-class DateConverter : public TypedConverter<arrow::DateBuilder> {
+class DateConverter : public TypedConverter<DateBuilder> {
  public:
   Status AppendData(PyObject* seq) override {
     Py_ssize_t size = PySequence_Size(seq);
@@ -314,7 +309,7 @@ class DateConverter : public TypedConverter<arrow::DateBuilder> {
   }
 };
 
-class TimestampConverter : public TypedConverter<arrow::TimestampBuilder> {
+class TimestampConverter : public TypedConverter<TimestampBuilder> {
  public:
   Status AppendData(PyObject* seq) override {
     Py_ssize_t size = PySequence_Size(seq);
@@ -347,7 +342,7 @@ class TimestampConverter : public TypedConverter<arrow::TimestampBuilder> {
   }
 };
 
-class DoubleConverter : public TypedConverter<arrow::DoubleBuilder> {
+class DoubleConverter : public TypedConverter<DoubleBuilder> {
  public:
   Status AppendData(PyObject* seq) override {
     double val;
@@ -367,7 +362,7 @@ class DoubleConverter : public TypedConverter<arrow::DoubleBuilder> {
   }
 };
 
-class BytesConverter : public TypedConverter<arrow::BinaryBuilder> {
+class BytesConverter : public TypedConverter<BinaryBuilder> {
  public:
   Status AppendData(PyObject* seq) override {
     PyObject* item;
@@ -401,7 +396,7 @@ class BytesConverter : public TypedConverter<arrow::BinaryBuilder> {
   }
 };
 
-class UTF8Converter : public TypedConverter<arrow::StringBuilder> {
+class UTF8Converter : public TypedConverter<StringBuilder> {
  public:
   Status AppendData(PyObject* seq) override {
     PyObject* item;
@@ -433,7 +428,7 @@ class UTF8Converter : public TypedConverter<arrow::StringBuilder> {
   }
 };
 
-class ListConverter : public TypedConverter<arrow::ListBuilder> {
+class ListConverter : public TypedConverter<ListBuilder> {
  public:
   Status Init(const std::shared_ptr<ArrayBuilder>& builder) override;
 
@@ -483,10 +478,10 @@ std::shared_ptr<SeqConverter> GetConverter(const std::shared_ptr<DataType>& type
 
 Status ListConverter::Init(const std::shared_ptr<ArrayBuilder>& builder) {
   builder_ = builder;
-  typed_builder_ = static_cast<arrow::ListBuilder*>(builder.get());
+  typed_builder_ = static_cast<ListBuilder*>(builder.get());
 
   value_converter_ =
-      GetConverter(static_cast<arrow::ListType*>(builder->type().get())->value_type());
+      GetConverter(static_cast<ListType*>(builder->type().get())->value_type());
   if (value_converter_ == nullptr) {
     return Status::NotImplemented("value type not implemented");
   }
@@ -508,8 +503,7 @@ Status AppendPySequence(PyObject* obj, const std::shared_ptr<DataType>& type,
   return converter->AppendData(obj);
 }
 
-Status ConvertPySequence(
-    PyObject* obj, MemoryPool* pool, std::shared_ptr<arrow::Array>* out) {
+Status ConvertPySequence(PyObject* obj, MemoryPool* pool, std::shared_ptr<Array>* out) {
   std::shared_ptr<DataType> type;
   int64_t size;
   PyDateTime_IMPORT;
@@ -517,16 +511,17 @@ Status ConvertPySequence(
 
   // Handle NA / NullType case
   if (type->type == Type::NA) {
-    out->reset(new arrow::NullArray(size));
+    out->reset(new NullArray(size));
     return Status::OK();
   }
 
   // Give the sequence converter an array builder
   std::shared_ptr<ArrayBuilder> builder;
-  RETURN_NOT_OK(arrow::MakeBuilder(pool, type, &builder));
+  RETURN_NOT_OK(MakeBuilder(pool, type, &builder));
   RETURN_NOT_OK(AppendPySequence(obj, type, builder));
 
   return builder->Finish(out);
 }
 
-}  // namespace pyarrow
+}  // namespace py
+}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/python/src/pyarrow/adapters/builtin.h
----------------------------------------------------------------------
diff --git a/python/src/pyarrow/adapters/builtin.h b/python/src/pyarrow/adapters/builtin.h
index 0c863a5..2d45e67 100644
--- a/python/src/pyarrow/adapters/builtin.h
+++ b/python/src/pyarrow/adapters/builtin.h
@@ -27,27 +27,28 @@
 
 #include <arrow/type.h>
 
+#include "arrow/util/visibility.h"
+
 #include "pyarrow/common.h"
-#include "pyarrow/visibility.h"
 
 namespace arrow {
+
 class Array;
 class Status;
-}
 
-namespace pyarrow {
+namespace py {
 
-PYARROW_EXPORT arrow::Status InferArrowType(
+ARROW_EXPORT arrow::Status InferArrowType(
     PyObject* obj, int64_t* size, std::shared_ptr<arrow::DataType>* out_type);
 
-PYARROW_EXPORT arrow::Status AppendPySequence(PyObject* obj,
+ARROW_EXPORT arrow::Status AppendPySequence(PyObject* obj,
     const std::shared_ptr<arrow::DataType>& type,
     const std::shared_ptr<arrow::ArrayBuilder>& builder);
 
-PYARROW_EXPORT
-arrow::Status ConvertPySequence(
-    PyObject* obj, arrow::MemoryPool* pool, std::shared_ptr<arrow::Array>* out);
+ARROW_EXPORT
+Status ConvertPySequence(PyObject* obj, MemoryPool* pool, std::shared_ptr<Array>* out);
 
-}  // namespace pyarrow
+}  // namespace py
+}  // namespace arrow
 
 #endif  // PYARROW_ADAPTERS_BUILTIN_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/00df40ce/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
index e286ccc..e694e79 100644
--- a/python/src/pyarrow/adapters/pandas-test.cc
+++ b/python/src/pyarrow/adapters/pandas-test.cc
@@ -30,9 +30,8 @@
 #include "arrow/type.h"
 #include "pyarrow/adapters/pandas.h"
 
-using namespace arrow;
-
-namespace pyarrow {
+namespace arrow {
+namespace py {
 
 TEST(PandasConversionTest, TestObjectBlockWriteFails) {
   StringBuilder builder;
@@ -61,4 +60,5 @@ TEST(PandasConversionTest, TestObjectBlockWriteFails) {
   Py_END_ALLOW_THREADS;
 }
 
+}  // namespace py
 }  // namespace arrow