arrow-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From w...@apache.org
Subject arrow git commit: ARROW-1417: [Python] Allow more generic filesystem objects to be passed to ParquetDataset
Date Mon, 04 Sep 2017 23:38:13 GMT
Repository: arrow
Updated Branches:
  refs/heads/master b1e56a2f5 -> ec32013fd


ARROW-1417: [Python] Allow more generic filesystem objects to be passed to ParquetDataset

This way, the `ParquetDataset` accepts both `S3FileSystem` and `LocalFileSystem` objects as
they are used in `dask`. By using `issubclass`, external libraries may write their own FS
wrappers by inheriting from the arrow FS.

I tested the integration with dask and this will fix the issue blocking https://github.com/dask/dask/pull/2527

Author: fjetter <florian.jetter@blue-yonder.com>

Closes #1032 from fjetter/ARROW-1417 and squashes the following commits:

75f18a5 [fjetter] Remove isinstance check in _ensure_filesystem
302b644 [fjetter] Perform check for type object before issubclass
ed111c9 [fjetter] Allow more generic filesystems to be passed


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

Branch: refs/heads/master
Commit: ec32013fd6df35b051173f0e9aa8aa8833f1c819
Parents: b1e56a2
Author: fjetter <florian.jetter@blue-yonder.com>
Authored: Mon Sep 4 19:38:07 2017 -0400
Committer: Wes McKinney <wes.mckinney@twosigma.com>
Committed: Mon Sep 4 19:38:07 2017 -0400

----------------------------------------------------------------------
 python/pyarrow/parquet.py | 22 ++++++++++++++--------
 1 file changed, 14 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/arrow/blob/ec32013f/python/pyarrow/parquet.py
----------------------------------------------------------------------
diff --git a/python/pyarrow/parquet.py b/python/pyarrow/parquet.py
index 4bc56eb..5dabca9 100644
--- a/python/pyarrow/parquet.py
+++ b/python/pyarrow/parquet.py
@@ -16,13 +16,14 @@
 # under the License.
 
 import os
+import inspect
 import json
 
 import six
 
 import numpy as np
 
-from pyarrow.filesystem import FileSystem, LocalFileSystem
+from pyarrow.filesystem import FileSystem, LocalFileSystem, S3FSWrapper
 from pyarrow._parquet import (ParquetReader, FileMetaData,  # noqa
                               RowGroupMetaData, ParquetSchema,
                               ParquetWriter)
@@ -645,13 +646,18 @@ class ParquetDataset(object):
 
 
 def _ensure_filesystem(fs):
-    if not isinstance(fs, FileSystem):
-        if type(fs).__name__ == 'S3FileSystem':
-            from pyarrow.filesystem import S3FSWrapper
-            return S3FSWrapper(fs)
-        else:
-            raise IOError('Unrecognized filesystem: {0}'
-                          .format(type(fs)))
+    fs_type = type(fs)
+
+    # If the arrow filesystem was subclassed, assume it supports the full interface and return
it
+    if not issubclass(fs_type, FileSystem):
+        for mro in inspect.getmro(fs_type):
+            if mro.__name__ is 'S3FileSystem':
+                return S3FSWrapper(fs)
+            # In case its a simple LocalFileSystem (e.g. dask) use native arrow FS
+            elif mro.__name__ is 'LocalFileSystem':
+                return LocalFileSystem.get_instance()
+
+        raise IOError('Unrecognized filesystem: {0}'.format(fs_type))
     else:
         return fs
 


Mime
View raw message