[
https://issues.apache.org/jira/browse/HDFS-7878?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14368360#comment-14368360
]
Colin Patrick McCabe commented on HDFS-7878:
--------------------------------------------
Thanks, [~sershe]. I like version 3 a lot more. [~jingzhao], did you get a
chance to take a look?
{code}
+ /**
+ * Opens an FSDataInputStream with the indicated file ID.
+ * @param fileId the file ID to open
+ * @param bufferSize the size of the buffer to be used.
+ */
+ public FSDataInputStream open(long fileId, final int bufferSize) throws
IOException {
+ return open(DFSUtil.makeInodePath(fileId), bufferSize);
+ }
{code}
Maybe we can save this one for another change? I think we have too many
overloads of {{FileSystem#open}} already... there must be at least a dozen, and
it's not clear to users how each of them are different, unless they check the
javadoc closely. Plus they can already open {{/.reserved/.inodes/XYZ}} to get
the same functionality (or even more functionality since we can specify things
like the buffer size...)
{code}
public final FileStatus makeQualified(URI defaultUri, Path path) {
- return new FileStatus(getLen(), isDir(), getReplication(),
+ return new DFSFileStatus(getLen(), isDir(), getReplication(),
{code}
I think we should just add a Long field (which can be null) to FileStatus. We
don't have to (de)serialize it in FileStatus#write or FileStatus#read, since
it's an optional field.
> 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
> Attachments: HDFS-7878.01.patch, HDFS-7878.02.patch,
> HDFS-7878.03.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)