Skip to content
Merged
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
40 changes: 15 additions & 25 deletions ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
Original file line number Diff line number Diff line change
Expand Up @@ -3744,6 +3744,19 @@ public void recycleDirToCmPath(Path dataPath, boolean isPurge) throws HiveExcept
}
}

private static void deleteAndRename(FileSystem destFs, Path destFile, FileStatus srcStatus, Path destPath)
throws IOException {
if (destFs.exists(destFile)) {
// rename cannot overwrite non empty destination directory, so deleting the destination before renaming.
destFs.delete(destFile);
LOG.info("Deleting destination file" + destFile.toUri());
}
if(!destFs.rename(srcStatus.getPath(), destFile)) {
throw new IOException("rename for src path: " + srcStatus.getPath() + " to dest:"
+ destPath + " returned false");
}
}

//it is assumed that parent directory of the destf should already exist when this
//method is called. when the replace value is true, this method works a little different
//from mv command if the destf is a directory, it replaces the destf instead of moving under
Expand Down Expand Up @@ -3829,37 +3842,14 @@ public static boolean moveFile(final HiveConf conf, Path srcf, final Path destf,
"Unable to move source " + srcStatus.getPath() + " to destination " + destFile;

if (null == pool) {
boolean success = false;
if (destFs instanceof DistributedFileSystem) {
((DistributedFileSystem)destFs).rename(srcStatus.getPath(), destFile, Options.Rename.OVERWRITE);
success = true;
} else {
destFs.delete(destFile, false);
success = destFs.rename(srcStatus.getPath(), destFile);
}
if(!success) {
throw new IOException("rename for src path: " + srcStatus.getPath() + " to dest:"
+ destf + " returned false");
}
deleteAndRename(destFs, destFile, srcStatus, destf);
} else {
futures.add(pool.submit(new Callable<Void>() {
@Override
public Void call() throws HiveException {
SessionState.setCurrentSessionState(parentSession);
try {
boolean success = false;
if (destFs instanceof DistributedFileSystem) {
((DistributedFileSystem)destFs).rename(srcStatus.getPath(), destFile, Options.Rename.OVERWRITE);
success = true;
} else {
destFs.delete(destFile, false);
success = destFs.rename(srcStatus.getPath(), destFile);
}
if (!success) {
throw new IOException(
"rename for src path: " + srcStatus.getPath() + " to dest path:"
+ destFile + " returned false");
}
deleteAndRename(destFs, destFile, srcStatus, destf);
} catch (Exception e) {
throw getHiveException(e, poolMsg);
}
Expand Down