[
https://issues.apache.org/jira/browse/HDFS-7878?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15504298#comment-15504298
]
Chris Douglas commented on HDFS-7878:
-------------------------------------
bq. How about using name service id ("dfs.nameservice.id") here?
Tracing through {{DFSClient}} init is quite a journey. Isn't the nameservice ID
passed as the URI for the client? Is setting it as in v06 insufficient?
The {{FileSystem}} implementations I looked at often return nonce information
as a String.
[S3a|http://docs.aws.amazon.com/AmazonS3/latest/API/RESTCommonResponseHeaders.html]
could check ETag or version ID in the response (both Strings),
[Azure|https://msdn.microsoft.com/en-us/library/azure/dd179371.aspx] blobs
could use Etag or DateTime (easy to encode), and HDFS needs to convert the
inodeId to a Path segment, anyway. That said: Strings in the JVM aren't an
efficient representation (particularly of HDFS inodes), some implementations
may not return sufficient metadata to generate a {{FileHandle}} from
{{FileStatus}} (requiring an RPC, so {{FileStatus}} would need a back-pointer
to its {{FileSystem}}), and requiring {{toString()}} serialization is
regrettable.
We could add {{byte[] FileSystem::createFileHandle(FileStatus)}} (or similar
variants), with the contract that these are the minimum set of bytes for a
[comparably configured] {{FileSystem}} instance to address exactly that inode.
This seems redundant with all the existing serialization, and most of the APIs
would be awkward (e.g., {{open(Path p, byte[] nonce)}} ?).
[~sershe], is it important to maintain {{open(FileHandle)}} independent of the
{{FileStatus}} instance? If Hive were to serialize the {{FileStatus}} instance
(with {{FileHandle}}) instead of just the path/inode, then it could use this
API. Adding other {{FileSystem}} operations accepting {{FileStatus}} also has
the virtue of reusing the most-commonly used {{getFileStatus}} and
{{listFileStatus}}, rather than another set of APIs managing {{FileHandle}}.
Put more directly, if we were to add directory listing (from a dirent, similar
to {{ftw/nftw}}), delete, rename, etc. we would probably not want to add these
for an opaque {{FileHandle}} reference (to which the caller would have to
retain a map).
Many applications perform checks on ownership, last-modification time, and
other metadata in race with {{FileSystem}} operations; AFAIK it's comparatively
rare that users would prefer that their operation to apply to whatever entity
is referenced by a {{Path}} at that moment. Point being: even if the
{{FileStatus}}-oriented APIs were used thoughtlessly, I doubt users would be
surprised at the semantics.
Instead of adding {{FileHandle}} as a {{Writable}}, perhaps this should take
HDFS-6984 as a prerequisite. I'm not sure why {{HdfsFileStatus}} doesn't extend
{{FileStatus}}. Is that something we could change in 3.x?
Aside: Java has already taken {{FileDescriptor}} and recently {{Path}}; should
this be a {{PathHandle}}?
Ping [[email protected]], [~cnauroth]
> API - expose an unique file identifier
> --------------------------------------
>
> Key: HDFS-7878
> URL: https://issues.apache.org/jira/browse/HDFS-7878
> Project: Hadoop HDFS
> Issue Type: Improvement
> Reporter: Sergey Shelukhin
> Assignee: Sergey Shelukhin
> Labels: BB2015-05-TBR
> Attachments: HDFS-7878.01.patch, HDFS-7878.02.patch,
> HDFS-7878.03.patch, HDFS-7878.04.patch, HDFS-7878.05.patch,
> HDFS-7878.06.patch, HDFS-7878.patch
>
>
> See HDFS-487.
> Even though that is resolved as duplicate, the ID is actually not exposed by
> the JIRA it supposedly duplicates.
> INode ID for the file should be easy to expose; alternatively ID could be
> derived from block IDs, to account for appends...
> This is useful e.g. for cache key by file, to make sure cache stays correct
> when file is overwritten.
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]