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 818a1c7af5ba..65602227aad4 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 @@ -89,6 +89,7 @@ import org.apache.commons.lang3.ObjectUtils; import org.apache.commons.lang3.tuple.Pair; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileChecksum; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -5068,15 +5069,37 @@ private static Path mvFile(HiveConf conf, FileSystem sourceFs, Path sourcePath, * * I'll leave the below loop for now until a better approach is found. */ - for (int counter = 1; destFs.exists(destFilePath); counter++) { - if (isOverwrite) { - destFs.delete(destFilePath, false); - break; - } - destFilePath = new Path(destDirPath, name + (Utilities.COPY_KEYWORD + counter) + + for (int counter = 1; counter < Integer.MAX_VALUE; counter++) { + if (destFs.exists(destFilePath)) { + // if exists and overwrite, simply delete and try to move + if (isOverwrite) { + destFs.delete(destFilePath, false); + } else { + // if exists and no overwrite, let's try to write to $original_destination_filename + "_copy_n" + destFilePath = new Path(destDirPath, name + (Utilities.COPY_KEYWORD + counter) + ((taskId == -1 && !type.isEmpty()) ? "." + type : "")); + } + } + try { + return doMvFile(conf, sourceFs, sourcePath, destFs, destFilePath, isSrcLocal, isRenameAllowed); + } catch (FileAlreadyExistsException faee) { + // There is a chance that the file did not exist at the time of the initial check, + // but it exists by the time this method attempts to write it. + // In that case, simply continue iterating. + LOG.warn("Destination file already exists while trying to move (attempt #{}): {} retrying...", + counter, faee.getMessage()); + } } + // It's unlikely that we reach this point because: + // a) The loop would have returned after a successful move (mv) operation, or + // b) An exception occurred (other than FileAlreadyExistsException). + // Therefore, we can only end up here if the file move was attempted Integer.MAX_VALUE times, + // and each attempt failed due to a FileAlreadyExistsException. + throw new RuntimeException("All file move operations failed to destination: " + destFilePath); + } + private static Path doMvFile(HiveConf conf, FileSystem sourceFs, Path sourcePath, FileSystem destFs, + Path destFilePath, boolean isSrcLocal, boolean isRenameAllowed) throws IOException { if (isRenameAllowed) { destFs.rename(sourcePath, destFilePath); } else if (isSrcLocal) {