diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java index cf2c3bc..c377716 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/spark/SparkUtilities.java @@ -22,12 +22,11 @@ import java.net.URI; import java.net.URISyntaxException; import java.util.Collection; -import java.util.UUID; -import java.util.Collection; import com.google.common.base.Preconditions; import org.apache.commons.io.FilenameUtils; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileAlreadyExistsException; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; @@ -91,11 +90,15 @@ public static URI getURI(String path) throws URISyntaxException { */ public static URI uploadToHDFS(URI source, HiveConf conf) throws IOException { Path localFile = new Path(source.getPath()); - // give the uploaded file a UUID - Path remoteFile = new Path(SessionState.getHDFSSessionPath(conf), - UUID.randomUUID() + "-" + getFileName(source)); + Path remoteFile = new Path(SessionState.getHDFSSessionPath(conf), getFileName(source)); FileSystem fileSystem = FileSystem.get(conf); - fileSystem.copyFromLocalFile(localFile, remoteFile); + if (fileSystem.exists(remoteFile)) { + // If we get here, it means user added two files from different paths but + // with same file name. Since we shouldn't rename the file, let's just + // throw exceptions on this. See HIVE-12229 + throw new FileAlreadyExistsException(remoteFile + " already exists."); + } + fileSystem.copyFromLocalFile(false, false, localFile, remoteFile); Path fullPath = fileSystem.getFileStatus(remoteFile).getPath(); return fullPath.toUri(); }