-
Notifications
You must be signed in to change notification settings - Fork 9.1k
HDFS-15484. Add new method batchRename for DistributedFileSystem and W… #2235
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
base: trunk
Are you sure you want to change the base?
Conversation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I've discussed some of what I'd like in https://issues.apache.org/jira/browse/HDFS-15484?focusedCommentId=17162752&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-17162752
- matching hadoop common JIRA to make clear this is going near the FS APIs
- something in the fileysystem spec to cover the new interface, define its semantics, etc. In particular, we need all things which are "obvious" to be written down, because often it turns out they aren't obvious at all.
- tests which really set out to break things. Writing the spec will help you think of them
Some ideas for tests
- renaming root
- rename to root
- rename to self
- path under self
- path above self
- two sources to same dest
- chained rename
- swapping paths
API wise, this could be our chance to fix rename properly, that is: I should be able to use this for a single rename((src, dest), opts) and have it do what I want. And as discussed, I Want something which works well with object stores
- use a builder to let apps specify options (see openFile()) and use the same base builder classes
- Return a future of the outcome. If we can get the HADOOP-16830 IOStatistics patch in first, the outome returned can be declared as it something which implement IOStatisticSource. This matters to me, as I want to know the costs of rename operations.
I think we should also add a rename option about atomicity; for a single rename() this would be that the rename itself is atomic. For a batch with size > 1, this means "the entire set of renames are atomic".
FileContext and ViewFS will also need to pass this through. Sorry.
One thing we could do here is actually provide a base implementation which iterates through the list/array of (src, dest) paths. This would let us add a non-atomic implementation to all filesystems/filecontexts. That would be very nice as it really would let me switch to using this API wherever we used rename(), such as distcp and MR committers.
rename() is that the trickiest of all FS API calls to get right. I don't think we fully understand what right is. certainly if I was asked about the nuances (src = file, dest = dir) and (src = dir, dest=dir) I'm not confident I could give an answer which is consistent for both POSIX and HDFS. This is our opportunity to make some progress here!
I know, this is going to add more work. But it is time.
* Rename a batch files or directories. | ||
* @see ClientProtocol#batchRename(String[] , String[], Options.Rename...) | ||
*/ | ||
public void batchRename(String[] srcs, String[] dsts, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
better as a list of <src, dest> pairs, so it's obvious about the mapping. Add javadocs
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK.
|
||
public BatchOpsException(long index, long total, | ||
String cause) { | ||
super("Batch operation break! " + |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
not sure about "break!"
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
change to "Batch operation partial success"
@InterfaceAudience.Private | ||
@InterfaceStability.Evolving | ||
public final class BatchOpsException extends IOException { | ||
private static final long serialVersionUID = 1L; |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
needs a real serial version ID; your IDE can help there
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done, Thanks!
|
||
public BatchOpsException(long index, long total, Throwable cause) { | ||
this(index, total, | ||
cause.getClass().getName() + ": " + cause.getMessage()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
cause.toString(); message may be null
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
add a more check.
@@ -139,4 +139,11 @@ private CommonPathCapabilities() { | |||
public static final String FS_MULTIPART_UPLOADER = | |||
"fs.capability.multipart.uploader"; | |||
|
|||
/** | |||
* Does the store support multipart uploading? |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
fix
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done
Path p = new Path(dir, tag + "_" + i); | ||
if (createNum-- > 0) { | ||
DFSTestUtil.createFile(dfs, p, 10, (short) 1, 0); | ||
assertTrue(dfs.exists(p)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ContractTestUtils assertFile() (or similar)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done, thanks!
dsts.toArray(new String[dsts.size()])); | ||
} catch (BatchOpsException e) { | ||
long index = e.getIndex(); | ||
assertEquals(1, index); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
all tests to add a message about what a failure means. Imagine you have a yetus test run failure and are trying to debug from the log only
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done.
|
||
@InterfaceAudience.Public | ||
@InterfaceStability.Unstable | ||
public interface BatchOperations { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
BatchRename
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I change to BatchRename, I thought we may add other batch method in this interface in the future.
* @param dsts target file list. | ||
* @throws IOException failure exception. | ||
*/ | ||
void batchRename(String[] srcs, String[] dsts, Options.Rename... options) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Prefer list or array of pairs. We don't have any pair type in hadoop here and can't use commons-lang as we don't want that in our public API. Maybe we should add one to org.apache.hadoop.common.utils and use it here amongst other places. I could certainly use it (I may be able to add this to HADOOP-16830 for you to pick up)
Return a future where we define RenameResult as something (class/interface) which implements IOStatisticsSource.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
OK, I'll integrate this later.
{ | ||
validateOpParams(op, destination); | ||
final EnumSet<Options.Rename> s = renameOptions.getValue(); | ||
cp.batchRename(fullpath.split(":"), destination.getValue().split(":"), |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
what will be done here if there's a ":" in paths
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The ":" is invalid character for hadoop absolute path, please refer to the method DFSUtilClient.isValidName().
Thanks @steveloughran for the detailed introduction. |
💔 -1 overall
This message was automatically generated. |
Updated #743 with a better (still WiP) attempt to factor out rename/3 in FileSystem and let implementations use efficiently. Once in we can just make rename/3 public there and see what subset of the (new) test we will need.
While we do this work in parallel (and me in my spare time :), you can create a rename.md file there too, just with your section....we can resolve the conflict when the time comes. |
…ebHdfsFileSystem
NOTICE
Please create an issue in ASF JIRA before opening a pull request,
and you need to set the title of the pull request which starts with
the corresponding JIRA issue number. (e.g. HADOOP-XXXXX. Fix a typo in YYY.)
For more details, please see https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute