You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2017/04/27 15:19:08 UTC

arrow git commit: ARROW-867: [Python] pyarrow MSVC fixes

Repository: arrow
Updated Branches:
  refs/heads/master e876abbdf -> 909f826b5


ARROW-867: [Python] pyarrow MSVC fixes

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

Closes #575 from wesm/ARROW-867 and squashes the following commits:

0483cfb [Wes McKinney] Do not encode file paths to utf-16le on Windows. Fix date/time conversion, platform ints. Add release/acquire methods to PyAcquireGIL lock object. Remove a couple unneeded GIL acquisitions


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

Branch: refs/heads/master
Commit: 909f826b55973e93f4656c43a84c8e740a86601f
Parents: e876abb
Author: Wes McKinney <we...@twosigma.com>
Authored: Thu Apr 27 11:19:02 2017 -0400
Committer: Wes McKinney <we...@twosigma.com>
Committed: Thu Apr 27 11:19:02 2017 -0400

----------------------------------------------------------------------
 ci/msvc-build.bat                           |  5 ++
 cpp/src/arrow/python/builtin_convert.cc     | 16 +----
 cpp/src/arrow/python/common.h               | 22 ++++++-
 cpp/src/arrow/python/helpers.cc             |  2 -
 cpp/src/arrow/python/init.cc                | 12 +---
 cpp/src/arrow/python/init.h                 | 12 +---
 cpp/src/arrow/python/numpy_interop.h        |  1 +
 cpp/src/arrow/python/pandas_convert.cc      | 17 +++--
 cpp/src/arrow/python/util/datetime.h        | 23 +++++++
 cpp/src/arrow/python/util/test_main.cc      |  2 +-
 python/pyarrow/_array.pyx                   |  6 +-
 python/pyarrow/_config.pyx                  |  6 +-
 python/pyarrow/_table.pyx                   |  2 +-
 python/pyarrow/compat.py                    | 11 +---
 python/pyarrow/includes/pyarrow.pxd         |  6 +-
 python/pyarrow/tests/test_convert_pandas.py |  4 +-
 python/pyarrow/tests/test_feather.py        |  5 ++
 python/pyarrow/tests/test_io.py             | 83 +++++++++++-------------
 python/pyarrow/tests/test_schema.py         |  1 -
 python/pyarrow/tests/test_tensor.py         | 32 ++++-----
 20 files changed, 140 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/ci/msvc-build.bat
----------------------------------------------------------------------
diff --git a/ci/msvc-build.bat b/ci/msvc-build.bat
index de428b6..08c5033 100644
--- a/ci/msvc-build.bat
+++ b/ci/msvc-build.bat
@@ -43,6 +43,8 @@ set PYTHONPATH=%CONDA_ENV%\Lib;%CONDA_ENV%\Lib\site-packages;%CONDA_ENV%\python3
 
 ctest -VV  || exit /B
 
+set PYTHONPATH=
+
 @rem Build and import pyarrow
 
 set PATH=%ARROW_HOME%\bin;%PATH%
@@ -50,3 +52,6 @@ set PATH=%ARROW_HOME%\bin;%PATH%
 cd ..\..\python
 python setup.py build_ext --inplace  || exit /B
 python -c "import pyarrow"  || exit /B
+
+@rem TODO: re-enable when last tests are fixed
+@rem py.test pyarrow -v -s || exit /B

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/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
index 137937c..3197c2a 100644
--- a/cpp/src/arrow/python/builtin_convert.cc
+++ b/cpp/src/arrow/python/builtin_convert.cc
@@ -358,22 +358,8 @@ class TimestampConverter : public TypedConverter<TimestampBuilder> {
       } 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);
+        typed_builder_->Append(PyDateTime_to_us(pydatetime));
         RETURN_IF_PYERROR();
-        struct tm epoch = {0};
-        epoch.tm_year = 70;
-        epoch.tm_mday = 1;
-        // Microseconds since the epoch
-        int64_t val = static_cast<int64_t>(
-            lrint(difftime(mktime(&datetime), mktime(&epoch))) * 1000000 + us);
-        typed_builder_->Append(val);
       }
     }
     return Status::OK();

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/cpp/src/arrow/python/common.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/common.h b/cpp/src/arrow/python/common.h
index 882bb15..0211823 100644
--- a/cpp/src/arrow/python/common.h
+++ b/cpp/src/arrow/python/common.h
@@ -34,11 +34,29 @@ namespace py {
 
 class ARROW_EXPORT PyAcquireGIL {
  public:
-  PyAcquireGIL() { state_ = PyGILState_Ensure(); }
+  PyAcquireGIL() : acquired_gil_(false) {
+    acquire();
+  }
+
+  ~PyAcquireGIL() { release(); }
 
-  ~PyAcquireGIL() { PyGILState_Release(state_); }
+  void acquire() {
+    if (!acquired_gil_) {
+      state_ = PyGILState_Ensure();
+      acquired_gil_ = true;
+    }
+  }
+
+  // idempotent
+  void release() {
+    if (acquired_gil_) {
+      PyGILState_Release(state_);
+      acquired_gil_ = false;
+    }
+  }
 
  private:
+  bool acquired_gil_;
   PyGILState_STATE state_;
   DISALLOW_COPY_AND_ASSIGN(PyAcquireGIL);
 };

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/cpp/src/arrow/python/helpers.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/helpers.cc b/cpp/src/arrow/python/helpers.cc
index f7c73a8..e5d1d38 100644
--- a/cpp/src/arrow/python/helpers.cc
+++ b/cpp/src/arrow/python/helpers.cc
@@ -55,7 +55,6 @@ std::shared_ptr<DataType> GetPrimitiveType(Type::type type) {
 }
 
 Status ImportModule(const std::string& module_name, OwnedRef* ref) {
-  PyAcquireGIL lock;
   PyObject* module = PyImport_ImportModule(module_name.c_str());
   RETURN_IF_PYERROR();
   ref->reset(module);
@@ -66,7 +65,6 @@ Status ImportFromModule(const OwnedRef& module, const std::string& name, OwnedRe
   /// Assumes that ImportModule was called first
   DCHECK_NE(module.obj(), nullptr) << "Cannot import from nullptr Python module";
 
-  PyAcquireGIL lock;
   PyObject* attr = PyObject_GetAttrString(module.obj(), name.c_str());
   RETURN_IF_PYERROR();
   ref->reset(attr);

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/cpp/src/arrow/python/init.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/init.cc b/cpp/src/arrow/python/init.cc
index fa70af7..db64891 100644
--- a/cpp/src/arrow/python/init.cc
+++ b/cpp/src/arrow/python/init.cc
@@ -15,20 +15,12 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "arrow/python/platform.h"
-
 // Trigger the array import (inversion of NO_IMPORT_ARRAY)
 #define NUMPY_IMPORT_ARRAY
 
 #include "arrow/python/init.h"
 #include "arrow/python/numpy_interop.h"
 
-namespace arrow {
-namespace py {
-
-void InitNumPy() {
-  import_numpy();
+int arrow_init_numpy() {
+  return arrow::py::import_numpy();
 }
-
-}  // namespace py
-}  // namespace arrow

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/cpp/src/arrow/python/init.h
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/init.h b/cpp/src/arrow/python/init.h
index a2533d8..1daa5a3 100644
--- a/cpp/src/arrow/python/init.h
+++ b/cpp/src/arrow/python/init.h
@@ -19,17 +19,11 @@
 #define ARROW_PYTHON_INIT_H
 
 #include "arrow/python/platform.h"
-
-#include "arrow/python/numpy_interop.h"
 #include "arrow/util/visibility.h"
 
-namespace arrow {
-namespace py {
-
+extern "C" {
 ARROW_EXPORT
-void InitNumPy();
-
-}  // namespace py
-}  // namespace arrow
+int arrow_init_numpy();
+}
 
 #endif  // ARROW_PYTHON_INIT_H

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/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
index b93200c..023fdc8 100644
--- a/cpp/src/arrow/python/numpy_interop.h
+++ b/cpp/src/arrow/python/numpy_interop.h
@@ -47,6 +47,7 @@ namespace py {
 
 inline int import_numpy() {
 #ifdef NUMPY_IMPORT_ARRAY
+  std::cout << "Importing NumPy" << std::endl;
   import_array1(-1);
   import_umath1(-1);
 #endif

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/cpp/src/arrow/python/pandas_convert.cc
----------------------------------------------------------------------
diff --git a/cpp/src/arrow/python/pandas_convert.cc b/cpp/src/arrow/python/pandas_convert.cc
index 9f65af4..b54197e 100644
--- a/cpp/src/arrow/python/pandas_convert.cc
+++ b/cpp/src/arrow/python/pandas_convert.cc
@@ -763,11 +763,10 @@ Status PandasConverter::ConvertObjects() {
 
   Ndarray1DIndexer<PyObject*> objects;
 
-  {
-    PyAcquireGIL lock;
-    objects.Init(arr_);
-    PyDateTime_IMPORT;
-  }
+  PyAcquireGIL lock;
+  objects.Init(arr_);
+  PyDateTime_IMPORT;
+  lock.release();
 
   // This means we received an explicit type from the user
   if (type_) {
@@ -792,6 +791,9 @@ Status PandasConverter::ConvertObjects() {
         return Status::TypeError("No known conversion to Arrow type");
     }
   } else {
+    // Re-acquire GIL
+    lock.acquire();
+
     OwnedRef decimal;
     OwnedRef Decimal;
     RETURN_NOT_OK(ImportModule("decimal", &decimal));
@@ -2196,7 +2198,12 @@ class ArrowDeserializer {
     RETURN_IF_PYERROR();
 
     PyObject* dictionary;
+
+    // Release GIL before calling ConvertArrayToPandas, will be reacquired
+    // there if needed
+    lock.release();
     RETURN_NOT_OK(ConvertArrayToPandas(dict_type->dictionary(), nullptr, &dictionary));
+    lock.acquire();
 
     PyDict_SetItemString(result_, "indices", block->block_arr());
     PyDict_SetItemString(result_, "dictionary", dictionary);

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/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
index ad0ee0f..7ebf46a 100644
--- a/cpp/src/arrow/python/util/datetime.h
+++ b/cpp/src/arrow/python/util/datetime.h
@@ -42,6 +42,29 @@ static inline int64_t PyDate_to_ms(PyDateTime_Date* pydate) {
 #endif
 }
 
+static inline int64_t PyDateTime_to_us(PyDateTime_DateTime* pydatetime) {
+  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);
+  struct tm epoch = {0};
+  epoch.tm_year = 70;
+  epoch.tm_mday = 1;
+#ifdef _MSC_VER
+  // Microseconds since the epoch
+  const int64_t current_timestamp = static_cast<int64_t>(_mkgmtime64(&datetime));
+  const int64_t epoch_timestamp = static_cast<int64_t>(_mkgmtime64(&epoch));
+  return (current_timestamp - epoch_timestamp) * 1000000L + us;
+#else
+  return static_cast<int64_t>(
+      lrint(difftime(mktime(&datetime), mktime(&epoch))) * 1000000 + us);
+#endif
+}
+
 static inline int32_t PyDate_to_days(PyDateTime_Date* pydate) {
   return static_cast<int32_t>(PyDate_to_ms(pydate) / 86400000LL);
 }

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/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
index c24da40..efb4475 100644
--- a/cpp/src/arrow/python/util/test_main.cc
+++ b/cpp/src/arrow/python/util/test_main.cc
@@ -25,7 +25,7 @@ int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
 
   Py_Initialize();
-  arrow::py::InitNumPy();
+  arrow_init_numpy();
 
   int ret = RUN_ALL_TESTS();
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/python/pyarrow/_array.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/_array.pyx b/python/pyarrow/_array.pyx
index 658f4b3..f01cff6 100644
--- a/python/pyarrow/_array.pyx
+++ b/python/pyarrow/_array.pyx
@@ -1288,8 +1288,7 @@ cdef class Array:
 
         with nogil:
             check_status(
-                pyarrow.ConvertArrayToPandas(self.sp_array, <PyObject*> self,
-                                             &out))
+                pyarrow.ConvertArrayToPandas(self.sp_array, self, &out))
         return wrap_array_output(out)
 
     def to_pylist(self):
@@ -1326,8 +1325,7 @@ strides: {2}""".format(self.type, self.shape, self.strides)
         cdef:
             PyObject* out
 
-        check_status(pyarrow.TensorToNdarray(deref(self.tp), <PyObject*> self,
-                                             &out))
+        check_status(pyarrow.TensorToNdarray(deref(self.tp), self, &out))
         return PyObject_to_object(out)
 
     def equals(self, Tensor other):

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/python/pyarrow/_config.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/_config.pyx b/python/pyarrow/_config.pyx
index 2c1e6bf..e5fdbef 100644
--- a/python/pyarrow/_config.pyx
+++ b/python/pyarrow/_config.pyx
@@ -14,13 +14,13 @@
 # distutils: language = c++
 # cython: embedsignature = True
 
-cdef extern from 'arrow/python/init.h' namespace 'arrow::py':
-    void InitNumPy()
+cdef extern from 'arrow/python/init.h':
+    int arrow_init_numpy() except -1
 
 cdef extern from 'arrow/python/config.h' namespace 'arrow::py':
     void set_numpy_nan(object o)
 
-InitNumPy()
+arrow_init_numpy()
 
 import numpy as np
 set_numpy_nan(np.nan)

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/python/pyarrow/_table.pyx
----------------------------------------------------------------------
diff --git a/python/pyarrow/_table.pyx b/python/pyarrow/_table.pyx
index 599e046..223fe27 100644
--- a/python/pyarrow/_table.pyx
+++ b/python/pyarrow/_table.pyx
@@ -164,7 +164,7 @@ cdef class Column:
             PyObject* out
 
         check_status(pyarrow.ConvertColumnToPandas(self.sp_column,
-                                                   <PyObject*> self, &out))
+                                                   self, &out))
 
         return _pandas().Series(wrap_array_output(out), name=self.name)
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/python/pyarrow/compat.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/compat.py b/python/pyarrow/compat.py
index 8d15c4c..928a2c0 100644
--- a/python/pyarrow/compat.py
+++ b/python/pyarrow/compat.py
@@ -139,14 +139,9 @@ def encode_file_path(path):
     import os
     # Windows requires utf-16le encoding for unicode file names
     if isinstance(path, unicode_type):
-        if os.name == 'nt':
-            # try:
-            #     encoded_path = path.encode('ascii')
-            # except:
-            encoded_path = path.encode('utf-16le')
-        else:
-            # POSIX systems can handle utf-8
-            encoded_path = path.encode('utf-8')
+        # POSIX systems can handle utf-8. UTF8 is converted to utf16-le in
+        # libarrow
+        encoded_path = path.encode('utf-8')
     else:
         encoded_path = path
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/python/pyarrow/includes/pyarrow.pxd
----------------------------------------------------------------------
diff --git a/python/pyarrow/includes/pyarrow.pxd b/python/pyarrow/includes/pyarrow.pxd
index c40df3d..35c7110 100644
--- a/python/pyarrow/includes/pyarrow.pxd
+++ b/python/pyarrow/includes/pyarrow.pxd
@@ -47,14 +47,14 @@ cdef extern from "arrow/python/api.h" namespace "arrow::py" nogil:
     CStatus NdarrayToTensor(CMemoryPool* pool, object ao,
                             shared_ptr[CTensor]* out);
 
-    CStatus TensorToNdarray(const CTensor& tensor, PyObject* base,
+    CStatus TensorToNdarray(const CTensor& tensor, object base,
                             PyObject** out)
 
     CStatus ConvertArrayToPandas(const shared_ptr[CArray]& arr,
-                                 PyObject* py_ref, PyObject** out)
+                                 object py_ref, PyObject** out)
 
     CStatus ConvertColumnToPandas(const shared_ptr[CColumn]& arr,
-                                  PyObject* py_ref, PyObject** out)
+                                  object py_ref, PyObject** out)
 
     CStatus ConvertTableToPandas(const shared_ptr[CTable]& table,
                                  int nthreads, PyObject** out)

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/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 2779da3..9b9b751 100644
--- a/python/pyarrow/tests/test_convert_pandas.py
+++ b/python/pyarrow/tests/test_convert_pandas.py
@@ -155,8 +155,8 @@ class TestPandasConversion(unittest.TestCase):
 
         for dtype, arrow_dtype in numpy_dtypes:
             info = np.iinfo(dtype)
-            values = np.random.randint(info.min,
-                                       min(info.max, np.iinfo('i8').max),
+            values = np.random.randint(max(info.min, np.iinfo(np.int_).min),
+                                       min(info.max, np.iinfo(np.int_).max),
                                        size=num_values)
             data[dtype] = values.astype(dtype)
             fields.append(pa.field(dtype, arrow_dtype))

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/python/pyarrow/tests/test_feather.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_feather.py b/python/pyarrow/tests/test_feather.py
index ef73a8f..7a8abf4 100644
--- a/python/pyarrow/tests/test_feather.py
+++ b/python/pyarrow/tests/test_feather.py
@@ -13,6 +13,7 @@
 # limitations under the License.
 
 import os
+import sys
 import unittest
 import pytest
 
@@ -251,6 +252,9 @@ class TestFeatherReader(unittest.TestCase):
         self._check_pandas_roundtrip(df, null_counts=[1 * repeats])
 
     def test_delete_partial_file_on_error(self):
+        if sys.platform == 'win32':
+            pytest.skip('Windows hangs on to file handle for some reason')
+
         # strings will fail
         df = pd.DataFrame(
             {
@@ -361,6 +365,7 @@ class TestFeatherReader(unittest.TestCase):
 
     def test_overwritten_file(self):
         path = random_path()
+        self.test_files.append(path)
 
         num_values = 100
         np.random.seed(0)

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/python/pyarrow/tests/test_io.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_io.py b/python/pyarrow/tests/test_io.py
index a14898f..610dedc 100644
--- a/python/pyarrow/tests/test_io.py
+++ b/python/pyarrow/tests/test_io.py
@@ -230,12 +230,13 @@ def test_nativefile_write_memoryview():
 
 
 @pytest.fixture
-def sample_disk_data(request):
+def sample_disk_data(request, tmpdir):
     SIZE = 4096
     arr = np.random.randint(0, 256, size=SIZE).astype('u1')
     data = arr.tobytes()[:SIZE]
 
-    path = guid()
+    path = os.path.join(str(tmpdir), guid())
+
     with open(path, 'wb') as f:
         f.write(data)
 
@@ -298,68 +299,62 @@ def _try_delete(path):
         pass
 
 
-def test_memory_map_writer():
+def test_memory_map_writer(tmpdir):
     SIZE = 4096
     arr = np.random.randint(0, 256, size=SIZE).astype('u1')
     data = arr.tobytes()[:SIZE]
 
-    path = guid()
-    try:
-        with open(path, 'wb') as f:
-            f.write(data)
+    path = os.path.join(str(tmpdir), guid())
+    with open(path, 'wb') as f:
+        f.write(data)
 
-        f = pa.memory_map(path, mode='r+w')
+    f = pa.memory_map(path, mode='r+w')
 
-        f.seek(10)
-        f.write('peekaboo')
-        assert f.tell() == 18
+    f.seek(10)
+    f.write('peekaboo')
+    assert f.tell() == 18
 
-        f.seek(10)
-        assert f.read(8) == b'peekaboo'
+    f.seek(10)
+    assert f.read(8) == b'peekaboo'
 
-        f2 = pa.memory_map(path, mode='r+w')
+    f2 = pa.memory_map(path, mode='r+w')
 
-        f2.seek(10)
-        f2.write(b'booapeak')
-        f2.seek(10)
+    f2.seek(10)
+    f2.write(b'booapeak')
+    f2.seek(10)
 
-        f.seek(10)
-        assert f.read(8) == b'booapeak'
+    f.seek(10)
+    assert f.read(8) == b'booapeak'
 
-        # Does not truncate file
-        f3 = pa.memory_map(path, mode='w')
-        f3.write('foo')
+    # Does not truncate file
+    f3 = pa.memory_map(path, mode='w')
+    f3.write('foo')
 
-        with pa.memory_map(path) as f4:
-            assert f4.size() == SIZE
+    with pa.memory_map(path) as f4:
+        assert f4.size() == SIZE
 
-        with pytest.raises(IOError):
-            f3.read(5)
+    with pytest.raises(IOError):
+        f3.read(5)
 
-        f.seek(0)
-        assert f.read(3) == b'foo'
-    finally:
-        _try_delete(path)
+    f.seek(0)
+    assert f.read(3) == b'foo'
 
 
-def test_os_file_writer():
+def test_os_file_writer(tmpdir):
     SIZE = 4096
     arr = np.random.randint(0, 256, size=SIZE).astype('u1')
     data = arr.tobytes()[:SIZE]
 
-    path = guid()
-    try:
-        with open(path, 'wb') as f:
-            f.write(data)
+    path = os.path.join(str(tmpdir), guid())
+    with open(path, 'wb') as f:
+        f.write(data)
 
-        # Truncates file
-        f2 = pa.OSFile(path, mode='w')
-        f2.write('foo')
+    # Truncates file
+    f2 = pa.OSFile(path, mode='w')
+    f2.write('foo')
 
-        with pa.OSFile(path) as f3:
-            assert f3.size() == 3
+    with pa.OSFile(path) as f3:
+        assert f3.size() == 3
 
-        with pytest.raises(IOError):
-            f2.read(5)
-    finally:
-        _try_delete(path)
+    with pytest.raises(IOError):
+        f2.read(5)

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/python/pyarrow/tests/test_schema.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_schema.py b/python/pyarrow/tests/test_schema.py
index b3abc0f..2d98865 100644
--- a/python/pyarrow/tests/test_schema.py
+++ b/python/pyarrow/tests/test_schema.py
@@ -206,7 +206,6 @@ def test_schema_equals():
     ]
 
     sch1 = pa.schema(fields)
-    print(dir(sch1))
     sch2 = pa.schema(fields)
     assert sch1.equals(sch2)
 

http://git-wip-us.apache.org/repos/asf/arrow/blob/909f826b/python/pyarrow/tests/test_tensor.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/tests/test_tensor.py b/python/pyarrow/tests/test_tensor.py
index ec71735..b0924e3 100644
--- a/python/pyarrow/tests/test_tensor.py
+++ b/python/pyarrow/tests/test_tensor.py
@@ -77,41 +77,37 @@ def test_tensor_numpy_roundtrip(dtype_str, arrow_type):
 
 
 def _try_delete(path):
+    import gc
+    gc.collect()
     try:
         os.remove(path)
     except os.error:
         pass
 
 
-def test_tensor_ipc_roundtrip():
+def test_tensor_ipc_roundtrip(tmpdir):
     data = np.random.randn(10, 4)
     tensor = pa.Tensor.from_numpy(data)
 
-    path = 'pyarrow-tensor-ipc-roundtrip'
-    try:
-        mmap = pa.create_memory_map(path, 1024)
+    path = os.path.join(str(tmpdir), 'pyarrow-tensor-ipc-roundtrip')
+    mmap = pa.create_memory_map(path, 1024)
 
-        pa.write_tensor(tensor, mmap)
+    pa.write_tensor(tensor, mmap)
 
-        mmap.seek(0)
-        result = pa.read_tensor(mmap)
+    mmap.seek(0)
+    result = pa.read_tensor(mmap)
 
-        assert result.equals(tensor)
-    finally:
-        _try_delete(path)
+    assert result.equals(tensor)
 
 
-def test_tensor_ipc_strided():
+def test_tensor_ipc_strided(tmpdir):
     data = np.random.randn(10, 4)
     tensor = pa.Tensor.from_numpy(data[::2])
 
-    path = 'pyarrow-tensor-ipc-strided'
-    try:
-        with pytest.raises(ValueError):
-            mmap = pa.create_memory_map(path, 1024)
-            pa.write_tensor(tensor, mmap)
-    finally:
-        _try_delete(path)
+    path = os.path.join(str(tmpdir), 'pyarrow-tensor-ipc-strided')
+    with pytest.raises(ValueError):
+        mmap = pa.create_memory_map(path, 1024)
+        pa.write_tensor(tensor, mmap)
 
 
 def test_tensor_size():