-
Notifications
You must be signed in to change notification settings - Fork 379
Compatibility with the new Arrow FileSystem implementations #295
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Comments
Just to make sure I understand the goals / issues: we'd like Arrow's new datasets API to work well with fsspec filesystems so that users and libraries like Dask can read data from s3 / gcs / azure blob storage using the new datasets API. And the primary issue is that the new datasets API is interacting with the filesystem in C++, so we really do need a C++ filesystem thing. Naively, I'd expect something like https://issues.apache.org/jira/browse/ARROW-7102 to be the best solution. Then anyone can come along with an fsspec-based filesystem and pyarrow will have (a hopefully thin) wrapper around it to get the pieces of data it needs about the filesystem. I haven't looked closely at the new filesystem, but is it doing filesystem operations like listing files from C++? In that case, ARROW-7102 and https://issues.apache.org/jira/browse/ARROW-8766 start to sound very similar right? The C++ filesystem will need to list the files in a directory, and so it calls back to python, which calls to the underlying fsspec-based implementation? Initially, I'm not too worried about performance here. At least for cloud storage the overhead of network calls tends to dominate. you describe in https://issues.apache.org/jira/browse/ARROW-8766 to work well. And at least for things like s3fs, gcsfs, and adlfs I don't expect performance to be an issue. The network overhead tends to dominate most things. |
Yeah, maybe ARROW-7102 and ARROW-8766 are basically somewhat duplicative. But the ARROW-7102 issue was not very clearly specifying the actual idea of what you describe above (a C++ filesystem calling back to python to ask the fsspec-based implementation), so for this I opened ARROW-8766 to track this idea more specifically. So it might be you were thinking of that when you opened ARROW-7102, but now with the new filesystems, it would not be something simple as a To quote @pitrou from ARROW-7102:
It's the second point for which I opened ARROW-8766 , but the first point is also useful, and it was for this point that I referenced ARROW-7102 at the end of the top post (when talking about providing a fsspec compatible class that wraps a pyarrow.fs filesystem), but indeed the issue was not specifically about that. Sorry for the confusion here :) To try to clear up this situation, I opened ARROW-8780 specifically for the wrapping of |
Yes, right now it's indeed specifically for the Datasets API (but long term we want to use those new filesystems everywhere in So practically speaking: if Dask would adopt the Datasets API for Parquet IO (-> dask/dask#6174), then Dask needs to pass a
So here the user can only specify the path (and optional
The above is for Dask's use case, but also pandas is using fsspec-compatible file systems (pandas already uses s3fs, and might be switching to use fsspec more generally in the near future, xref pandas-dev/pandas#33549). And in pyarrow itself, we also currently accept fsspec filesystems in certain cases (and here as actual object, not parsed from a URI). For example, the top google search for reading parquet from s3 with python gives basically this solution:
This works right now because |
Thanks. I'd personally consider
to be a blocker to changing the From my point of view it seems natural for Dask / others to continue to pass an |
In the mean time, we have made progress on the "pyarrow.fs filesytem wrapping an fsspec-filesytem" option. Antoine implemented the low-level callback details, and I have now a PR that creates a "filesystem handler" based on an fsspec-compatible filesytem: apache/arrow#7395 Feedback is very welcome, and especially if someone could test it with an actual s3 or azure filesystem would be interesting (I currently only tested it with a local and in-memory filesystem, and a minio server) Now, that is for wrapping an fsspec filesystem. But I would also still be interested to get thoughts on actually converting an fsspec filesystem to a naive pyarrow filesystem (for the ones we support). Eg, would that be possible to do this robustly / faithfully for eg S3? Can all required information be encoded in an URI, for example, which could then be parsed by pyarrow? (that might be hard with all different ways to configure/store credentials?) |
Quick reply: minio is a good alternative to a "real" S3, I would trust it for testing; even moto is very nearly complete. The rest will take more thinking, but I doubt you could encode all the options for S3 in a URL. s3fs does support including the key/secret in the URL, but nothing else, and this is rarely used given the many other ways to provide credentials. |
@martindurant @jorisvandenbossche What about the other way round ? Wrapping the new arrow fs into fs spec ? I'm asking this because I'm (still) interested in having a generic, easy to use, FS which supports both HDFS & S3. What is currently existing with S3FsWrapper in pyarrow and the old pyarrow filesystem is nearly what I need. I also like the fsspec idea and I think the fsspec api is much easier to use than the new pyarrow fs api. What are your thoughts about this ? |
See the final note of the top post ("the above is about the use case of "users can pass an fsspec filesystem to pyarrow". There is also the use case the other way around of "using pyarrow filesystems where fsspec-compliant filesystems are expected" ... "). |
OK. Thanks. I will watch those tickets. It would be nice if this could follow the pattern that fsspec really only provides the interface and some helpers maybe (to avoid pulling in unnecessary dependencies in client projects) and then each project that provides the functionality pulls in fsspec and implements it. So to me this would mean arrow provides the S3/HDFS implementation, pulls in fsspec and then the client project pulls fsspec for the interfaces (+ maybe some helpers) + pyarrow + other libraries that provide other interesting fs implementations. |
That is exactly what I had in mind when writing fsspec :) |
Yes, currently I'm actually using S3FS + HDFS via arrow. It works pretty well. Thanks. |
Maybe arrow wants to do this, but s3fs will remain as a supported project with a large user base. I understand that arrow would like to implement in C++ to provide cross-language support, but the pydata ecosystem appreciates a simple pure-python package. |
It seems like the pyarrow.dataset feature at least partially supports fsspec and Azure Blob. After many guess and check testing attempts, I finally got this to work (using fsspec)
But if I try to read a partitioned dataset instead of a single file then I run into this error:
I couldn't find any documentation for Azure Blob for this new dataset feature but pq.ParquetDataset works fine, Dask and pandas works, and the DaskFileSystem seems to connect fine (I can list directories and all of that)
I just wonder if I am doing something wrong - it seems like I am so close right now and I want to experiment with non-Hive partitioning and the row filtering. |
@ldacey Can you report a bug to Arrow? See https://arrow.apache.org/docs/developers/contributing.html#report-bugs-and-propose-features for how to do this. |
@pitrou I edited the original issue I made about this subject when I thought that the dataset API would not work at all with Azure Blob. Now I know that it does work (a bit), but perhaps there is some conflict with fsspec and pyarrow filesystem since they both have
|
FYI - I reverted to 0.17.1 to test this again.
Reading a single parquet file no longer works with the
Dask seems to work either way. |
@jorisvandenbossche , are you still following this? |
@martindurant yes, is there anything specific to look at / answer to? (the recent comments regarding Azure are further discussed in https://issues.apache.org/jira/browse/ARROW-9514 I think?) |
Indeed - and it seems that was actually all fine. I think this can be closed. |
One additional aspect (but can also open a new issue for that), is that fsspec will need to update its hdfs filesystem implementation at some point (as right now it wraps the |
What is the deprecation timeline? I suggest a new issue when warnings start showing up. |
The timeline is that it will be actually deprecated (with warning) in pyarrow 2.0, to be released in a couple of weeks. |
So what is the status of the new implementation, can it be used with/from fsspec? |
Maybe you should replace it by https://github.com/dask/hdfs3. I was using this before and it was working great. |
hdfs3 used to be the way to do this (indeed, it was the very first of the fsspec-like implementations). However libhdfs3 (the C++ library it depends on) has proven difficult to maintain, and incomplete for secured connections, which I was not able to solve. |
We used it with kerberos authentification on our cluster if you mean that. Worked pretty well. The only thing that didn't work was viewfs. So we implemented on top on our own. But anyway nevermind the comment. I was a user of hdfs3 and really liked it, then I moved the old pyarrow fs. It is not that bad, I see the limitations now that I use it. But imo it was all fixable. |
I believe it was the line-encryption, not the auth, but old history now... Let's see what @jorisvandenbossche says about the new implementation. It would be great if it could simply be registered with fsspec and work. Then the shim we have here can indeed be dropped. |
As already explained multiple times in this thread: the
It's not only about HDFS, I think? Also the other filesystems inherit from
And libhdfs3 is also no longer being maintained, AFAIK. Also pyarrow dropped the optional libhdfs3 driver support and the new |
Background
@martindurant as you know, last year we started developing new FileSystem implementations in the Apache Arrow project (https://issues.apache.org/jira/browse/ARROW-767, apache/arrow#4225 is the PR with the initial abstract API, on which you gave feedback). Those developments have some implications for users using fsspec-compatible filesystems, and so as promised, with some delay, opening an issue here to discuss how to handle those implications (and since fsspec currently holds the pyarrow-compatibiliy layer, opening an issue here seems appropriate).
To summarize:
pyarrow.filesystems
(docs). We basically only have aLocalFileSystem
andpa.hdfs.HadoopFileSystem
as concrete implementations. And in addition, there is theDaskFileSystem
which is used by fsspec as base class, see more on that below.pyarrow.fs
submodule (docs). Those are python wrappers for the C++ implementations, and currently there are already concrete implementations for local, Hadoop and S3.So an important difference is that the new filesystems are actual implementations in C++, and
pyarrow.fs
is only providing wrappers for those. This is done for good reasons: those filesystem are a shared implementation and are used by many different users of the Arrow project (and from C, C++, Python, R, Ruby, ..). Further, those filesystems are for example used in the Arrow Datasets project, which enables a bunch of new features in the ParquetDataset reading (and also enabled that you can now actually query a Parquet dataset from R). Those new filesystems have been an important part in moving the Arrow project forward.But this also means that the filesystem that pyarrow functions expect is no longer an "interface" you can implement, but it actually needs a filesystem that wraps a C++ filesystem.
(to be clear: all functionality that already existed before is right now still accepting the old filesystems, only the new
pyarrow.dataset
module already requires the new filesystems. But long term, we want to move consistently to the new filesystems).Concretely, this means that the feature of
fsspec
to automatically provide compatibility with payrrow will no longer work in the future:This current compatibility means that eg pyarrow's
parquet.read_table/ParquetDataset
work with any fsspec filesystem.Concrete issues
Ideally, we want to keep compatibility for the existing user base that is using fsspec-based filesystems with pyarrow functionality, while at the same time internally in pyarrow moving completely to our new filesytem implementation.
To achieve this, I currently see two (not necessarily mutually exclusive, to be clear) options:
pyarrow.fs
filesystem object (eg convert as3fs.S3FileSystem
instance with all its configuration into an equivalentpyarrow.fs.S3FileSystem
).pyarrow.fs
wrapper forfsspec
", a C++ FileSystem that calls back into a python object for each of its methods (where this python object then could be any fsspec-compatible filesystem).There is actually also a third option, and that is that some concrete fsspec implementations start to use one of the new pyarrow filesystems as its base, and then it would also be directly usable in pyarrow (but that's not my call to make, but up to those individual projects to be clear. For HDFS in
fsspec
that's probably what we want to do, though, since its implementation already depends on pyarrow).As mentioned above, those options are not necessarily mutually exclusive. It might depend on the specific filesystem which option is desirable / possible (and the second option could also be a fallback for the first option if pyarrow doesn't support the specific file system).
Thoughts on this? About the feasibility of the specific options? Other options?
Note: the above is about the use case of "users can pass an fsspec filesystem to pyarrow". There is also the use case the other way around of "using pyarrow filesystems where fsspec-compliant filesystems are expected". For this, an
fsspec
-compliant wrapper around apyarrow.fs
filesystem is probably useful (and I suppose this is something that could live in pyarrow). For this there is https://issues.apache.org/jira/browse/ARROW-7102Such a wrapper could also provide a richer API for those users who want that with the pyarrow filesystems (since the current pyarrow.fs filesystems are rather bare-bones)
cc @martindurant @TomAugspurger @pitrou
The text was updated successfully, but these errors were encountered: