harshal-16 commented on code in PR #5990: URL: https://github.com/apache/hive/pull/5990#discussion_r2268717399
########## common/src/java/org/apache/hadoop/hive/common/FileUtils.java: ########## @@ -843,6 +843,9 @@ public static boolean copy(FileSystem srcFS, Path[] srcs, FileSystem dstFS, Path if (!doIOUtilsCopyBytes(srcFS, srcFS.getFileStatus(src), dstFS, dst, deleteSource, overwrite, preserveXAttr, conf, copyStatistics)) { returnVal = false; } + } catch (FileNotFoundException var15) { + // Wrapping the FileNotFoundException in a new IOException and re-throw immediately. + throw new IOException("Copy operation failed", var15); Review Comment: I didn't quite get your point here, Can please explain a bit more. While copy is going on, if files get cleaned from the source location (the most common scenario is the CM root getting cleaned), then it will throw FileNotFoundException, which is fatal, and there is no way the system will recover on its own like other IOException. Moreover, we may not want to throw all different exceptions; that's why we are wrapping all kinds of exceptions in IOException. That's what i am checking as part of ```java private boolean containsNonRetryableCause(Exception e) { Throwable current = e; while (current != null) { final Throwable curr = current; if (this.failOnParentExceptions.stream().anyMatch(k -> k.isAssignableFrom(curr.getClass()))) { return true; } current = current.getCause(); } return false; } ``` ########## ql/src/test/org/apache/hadoop/hive/ql/parse/repl/TestCopyUtils.java: ########## @@ -244,4 +261,91 @@ public void testParallelCopySuccess() throws Exception { Mockito.times(1)).invokeAll(callableCapture.capture()); } } + + @Test + public void testCopyFilesBetweenFSWithDestDirNotExistFailure() throws IOException { + Path srcPath1 = new Path(basePath, "file1.txt"); + Path srcPath2 = new Path(basePath, "file2.txt"); + Path dstPath = new Path(basePath, "copyDst"); + + try { + // Create source files + fileSystem.create(srcPath1).write("Content of file1".getBytes()); + fileSystem.create(srcPath2).write("Content of file2".getBytes()); Review Comment: ack, will update ########## ql/src/test/org/apache/hadoop/hive/ql/parse/repl/TestCopyUtils.java: ########## @@ -244,4 +261,91 @@ public void testParallelCopySuccess() throws Exception { Mockito.times(1)).invokeAll(callableCapture.capture()); } } + + @Test + public void testCopyFilesBetweenFSWithDestDirNotExistFailure() throws IOException { + Path srcPath1 = new Path(basePath, "file1.txt"); + Path srcPath2 = new Path(basePath, "file2.txt"); + Path dstPath = new Path(basePath, "copyDst"); + + try { + // Create source files + fileSystem.create(srcPath1).write("Content of file1".getBytes()); + fileSystem.create(srcPath2).write("Content of file2".getBytes()); + + // Prepare source paths array + Path[] srcPaths = {srcPath1, srcPath2}; + + + CopyUtils copyUtils = new CopyUtils("hive", hiveConf, fileSystem); + DataCopyStatistics copyStatistics = new DataCopyStatistics(); + IOException thrown = + Assert.assertThrows(IOException.class, () -> { + copyUtils.copyFilesBetweenFS(fileSystem, srcPaths, fileSystem, dstPath, false, true, copyStatistics); + }); + // this is supposed to come out of retryable function immediately without waiting as FileNotFound is not auto-recoverable error + Assert.assertEquals(IOException.class, thrown.getCause().getClass()); + Assert.assertEquals(FileNotFoundException.class, thrown.getCause().getCause().getClass()); + Assert.assertEquals("'/tmp/copyDst': specified destination directory does not exist", thrown.getCause().getMessage()); + + } finally { + // Clean up + fileSystem.delete(srcPath1, false); + fileSystem.delete(srcPath2, false); + fileSystem.delete(dstPath, true); + } + } + + @Test + public void testCopyFilesBetweenFSWithSourceFileGettingDeletedFailure() throws IOException { Review Comment: ack ########## ql/src/test/org/apache/hadoop/hive/ql/parse/repl/TestCopyUtils.java: ########## @@ -244,4 +261,91 @@ public void testParallelCopySuccess() throws Exception { Mockito.times(1)).invokeAll(callableCapture.capture()); } } + + @Test + public void testCopyFilesBetweenFSWithDestDirNotExistFailure() throws IOException { + Path srcPath1 = new Path(basePath, "file1.txt"); + Path srcPath2 = new Path(basePath, "file2.txt"); + Path dstPath = new Path(basePath, "copyDst"); + + try { + // Create source files + fileSystem.create(srcPath1).write("Content of file1".getBytes()); + fileSystem.create(srcPath2).write("Content of file2".getBytes()); + + // Prepare source paths array + Path[] srcPaths = {srcPath1, srcPath2}; + + + CopyUtils copyUtils = new CopyUtils("hive", hiveConf, fileSystem); + DataCopyStatistics copyStatistics = new DataCopyStatistics(); + IOException thrown = + Assert.assertThrows(IOException.class, () -> { + copyUtils.copyFilesBetweenFS(fileSystem, srcPaths, fileSystem, dstPath, false, true, copyStatistics); + }); + // this is supposed to come out of retryable function immediately without waiting as FileNotFound is not auto-recoverable error + Assert.assertEquals(IOException.class, thrown.getCause().getClass()); + Assert.assertEquals(FileNotFoundException.class, thrown.getCause().getCause().getClass()); + Assert.assertEquals("'/tmp/copyDst': specified destination directory does not exist", thrown.getCause().getMessage()); Review Comment: ack, will check ########## ql/src/test/org/apache/hadoop/hive/ql/parse/repl/TestCopyUtils.java: ########## @@ -244,4 +261,91 @@ public void testParallelCopySuccess() throws Exception { Mockito.times(1)).invokeAll(callableCapture.capture()); } } + + @Test + public void testCopyFilesBetweenFSWithDestDirNotExistFailure() throws IOException { + Path srcPath1 = new Path(basePath, "file1.txt"); + Path srcPath2 = new Path(basePath, "file2.txt"); + Path dstPath = new Path(basePath, "copyDst"); + + try { + // Create source files + fileSystem.create(srcPath1).write("Content of file1".getBytes()); + fileSystem.create(srcPath2).write("Content of file2".getBytes()); + + // Prepare source paths array + Path[] srcPaths = {srcPath1, srcPath2}; + + + CopyUtils copyUtils = new CopyUtils("hive", hiveConf, fileSystem); + DataCopyStatistics copyStatistics = new DataCopyStatistics(); + IOException thrown = + Assert.assertThrows(IOException.class, () -> { + copyUtils.copyFilesBetweenFS(fileSystem, srcPaths, fileSystem, dstPath, false, true, copyStatistics); + }); + // this is supposed to come out of retryable function immediately without waiting as FileNotFound is not auto-recoverable error + Assert.assertEquals(IOException.class, thrown.getCause().getClass()); + Assert.assertEquals(FileNotFoundException.class, thrown.getCause().getCause().getClass()); Review Comment: Yes, this works, as I am throwing error in FileUtils.java, which preserves the cause ```java } catch (FileNotFoundException var15) { // Wrapping the FileNotFoundException in a new IOException and re-throw immediately. throw new IOException("Copy operation failed", var15); ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: gitbox-unsubscr...@hive.apache.org For additional commands, e-mail: gitbox-h...@hive.apache.org