[ 
https://issues.apache.org/jira/browse/ARROW-9226?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17396856#comment-17396856
 ] 

Itamar Turner-Trauring commented on ARROW-9226:
-----------------------------------------------

Looking through the code—

**The deprecated API:**

1. `pyarrow.hdfs` imports from `_hdfsio.pyx`.
2. This is thin wrapper around `CIOHadoopFileSystem` and `HdfsConnectionConfig`.
3. The former is wrapper around `arrow::io::HadoopFileSystem` (see 
`libarrow.pxd`).
 
**The new API:**

1. `pyarrow.fs` imports from `_hdfs.pyx`
2. This builds on Cython classes `CHdfsOptions` and `CHadoopFileSystem`, with 
very small amount of wrapper code.
3. These are synonyms for C++ classes `arrow::fs::HdfsOptions` and 
`arrow::fs::HadoopFilesystem` (see `libarrow_fs.pxd`).

---

Looking through the old code, the connection code path is in 
`cpp/src/arrow/io/hdfs.cc`, and mostly interacts with the driver, which is in 
either from `libhdfs` or `libhdfs3` via a shim 
(https://github.com/apache/arrow/blob/7b66f97330215fe020ec536671ee50f41aa1af35/cpp/src/arrow/io/hdfs_internal.h),
 which `dlopen()`s the underlying `libhdfs` library at least.

... and `libhdfs` then calls into Java implementation.

Further digging suggests that the new code path (`arrow::fs::HadoopFilesystem`) 
still uses the `libhdfs/libhdfs3` drivers from `arrow::io`:

https://github.com/apache/arrow/blob/7b66f97330215fe020ec536671ee50f41aa1af35/cpp/src/arrow/filesystem/hdfs.cc#L56

Given all the heavy lifting seems to be done by the underlying libraries, this 
suggests this functionality could be exposed again, and the issue is less 
implementing the logic, and more just exposing the underlying API again.

I am probably going to try to do this.

> [Python] pyarrow.fs.HadoopFileSystem - retrieve options from core-site.xml or 
> hdfs-site.xml if available
> --------------------------------------------------------------------------------------------------------
>
>                 Key: ARROW-9226
>                 URL: https://issues.apache.org/jira/browse/ARROW-9226
>             Project: Apache Arrow
>          Issue Type: Improvement
>          Components: C++, Python
>    Affects Versions: 0.17.1
>            Reporter: Bruno Quinart
>            Priority: Minor
>              Labels: hdfs
>
> 'Legacy' pyarrow.hdfs.connect was somehow able to get the namenode info from 
> the hadoop configuration files.
> The new pyarrow.fs.HadoopFileSystem requires the host to be specified.
> Inferring this info from "the environment" makes it easier to deploy 
> pipelines.
> But more important, for HA namenodes it is almost impossible to know for sure 
> what to specify. If a rolling restart is ongoing, the namenode is changing. 
> There is no guarantee on which will be active in a HA setup.
> I tried connecting to the standby namenode. The connection gets established, 
> but when writing a file an error is raised that standby namenodes are not 
> allowed to write to.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to