This is an automated email from the ASF dual-hosted git repository. mahesh pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/hive.git
The following commit(s) were added to refs/heads/master by this push: new 5c425dc HIVE-21717 : Rename is failing for directory in move task. (Mahesh Kumar Behera reviewed by Sankar Hariappan 5c425dc is described below commit 5c425dca2fc95ed718d13513de7a7319e20c5261 Author: Mahesh Kumar Behera <mah...@apache.org> AuthorDate: Mon May 13 11:33:29 2019 +0530 HIVE-21717 : Rename is failing for directory in move task. (Mahesh Kumar Behera reviewed by Sankar Hariappan --- .../org/apache/hadoop/hive/ql/metadata/Hive.java | 40 ++++++++-------------- 1 file changed, 15 insertions(+), 25 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java index 55ae535..27ec7dc 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java @@ -4301,6 +4301,19 @@ private void constructOneLBLocationMap(FileStatus fSta, } } + 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 @@ -4386,37 +4399,14 @@ private void constructOneLBLocationMap(FileStatus fSta, "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); }