[ https://issues.apache.org/jira/browse/HIVE-26481?focusedWorklogId=803534&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-803534 ]
ASF GitHub Bot logged work on HIVE-26481: ----------------------------------------- Author: ASF GitHub Bot Created on: 25/Aug/22 09:29 Start Date: 25/Aug/22 09:29 Worklog Time Spent: 10m Work Description: ayushtkn commented on code in PR #3531: URL: https://github.com/apache/hive/pull/3531#discussion_r954708701 ########## common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java: ########## @@ -281,4 +279,32 @@ public void testMakeRelative() { relativePath = FileUtils.makeRelative(parentPath, childPath); assertEquals(childPath.toString(), relativePath.toString()); } + + @Test + public void testListStatusIterator() throws Exception { + + MockFileSystem fs = new MockFileSystem(new HiveConf(), + new MockFileSystem.MockFile("mock:/tmp/.staging", 500, new byte[0]), + new MockFileSystem.MockFile("mock:/tmp/_dummy", 500, new byte[0]), + new MockFileSystem.MockFile("mock:/tmp/dummy", 500, new byte[0])); + Path path = new MockFileSystem.MockPath(fs, "/tmp"); + RemoteIterator<FileStatus> it = FileUtils.listStatusIterator(fs, path, FileUtils.HIDDEN_FILES_PATH_FILTER); + Assert.assertEquals(1, getFileCount(it)); + RemoteIterator<LocatedFileStatus> itr = FileUtils.listFilesIterator(fs, path, true, FileUtils.HIDDEN_FILES_PATH_FILTER); + Assert.assertEquals(1, getFileCount(itr)); + } + + private int getFileCount(RemoteIterator<?> lfs) throws Exception { + try { + int count = 0; + while (lfs.hasNext()) { + lfs.next(); + count++; + } + return count; + } catch (IOException e) { + throw new Exception("Exception while list files on " + e); + } Review Comment: there is no need to catch & throw let the exception surface ########## common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java: ########## @@ -281,4 +279,32 @@ public void testMakeRelative() { relativePath = FileUtils.makeRelative(parentPath, childPath); assertEquals(childPath.toString(), relativePath.toString()); } + + @Test + public void testListStatusIterator() throws Exception { + + MockFileSystem fs = new MockFileSystem(new HiveConf(), + new MockFileSystem.MockFile("mock:/tmp/.staging", 500, new byte[0]), + new MockFileSystem.MockFile("mock:/tmp/_dummy", 500, new byte[0]), + new MockFileSystem.MockFile("mock:/tmp/dummy", 500, new byte[0])); + Path path = new MockFileSystem.MockPath(fs, "/tmp"); + RemoteIterator<FileStatus> it = FileUtils.listStatusIterator(fs, path, FileUtils.HIDDEN_FILES_PATH_FILTER); + Assert.assertEquals(1, getFileCount(it)); + RemoteIterator<LocatedFileStatus> itr = FileUtils.listFilesIterator(fs, path, true, FileUtils.HIDDEN_FILES_PATH_FILTER); + Assert.assertEquals(1, getFileCount(itr)); + } + + private int getFileCount(RemoteIterator<?> lfs) throws Exception { Review Comment: ? can be replaced by `? extends FileStatus`, LocatedFileStatus also extends from `FileStatus` ########## ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java: ########## @@ -1500,7 +1500,7 @@ public static Map<Path, HdfsDirSnapshot> getHdfsDirSnapshotsForCleaner(final Fil throws IOException { Map<Path, HdfsDirSnapshot> dirToSnapshots = new HashMap<>(); Deque<RemoteIterator<FileStatus>> stack = new ArrayDeque<>(); - stack.push(fs.listStatusIterator(path)); + stack.push(FileUtils.listStatusIterator(fs,path,acidHiddenFileFilter)); Review Comment: nit: missing space after comma. ########## common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java: ########## @@ -35,14 +35,12 @@ import java.util.HashSet; import java.util.Set; -import org.apache.hadoop.fs.ContentSummary; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocalFileSystem; -import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.*; Review Comment: don't club the imports. ########## common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java: ########## @@ -281,4 +279,32 @@ public void testMakeRelative() { relativePath = FileUtils.makeRelative(parentPath, childPath); assertEquals(childPath.toString(), relativePath.toString()); } + + @Test + public void testListStatusIterator() throws Exception { + + MockFileSystem fs = new MockFileSystem(new HiveConf(), + new MockFileSystem.MockFile("mock:/tmp/.staging", 500, new byte[0]), + new MockFileSystem.MockFile("mock:/tmp/_dummy", 500, new byte[0]), + new MockFileSystem.MockFile("mock:/tmp/dummy", 500, new byte[0])); + Path path = new MockFileSystem.MockPath(fs, "/tmp"); + RemoteIterator<FileStatus> it = FileUtils.listStatusIterator(fs, path, FileUtils.HIDDEN_FILES_PATH_FILTER); + Assert.assertEquals(1, getFileCount(it)); + RemoteIterator<LocatedFileStatus> itr = FileUtils.listFilesIterator(fs, path, true, FileUtils.HIDDEN_FILES_PATH_FILTER); + Assert.assertEquals(1, getFileCount(itr)); Review Comment: there is already a static import for assertEquals in the class, no need to prefix Assert. ########## common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java: ########## @@ -281,4 +279,32 @@ public void testMakeRelative() { relativePath = FileUtils.makeRelative(parentPath, childPath); assertEquals(childPath.toString(), relativePath.toString()); } + + @Test + public void testListStatusIterator() throws Exception { + + MockFileSystem fs = new MockFileSystem(new HiveConf(), + new MockFileSystem.MockFile("mock:/tmp/.staging", 500, new byte[0]), + new MockFileSystem.MockFile("mock:/tmp/_dummy", 500, new byte[0]), + new MockFileSystem.MockFile("mock:/tmp/dummy", 500, new byte[0])); + Path path = new MockFileSystem.MockPath(fs, "/tmp"); + RemoteIterator<FileStatus> it = FileUtils.listStatusIterator(fs, path, FileUtils.HIDDEN_FILES_PATH_FILTER); + Assert.assertEquals(1, getFileCount(it)); + RemoteIterator<LocatedFileStatus> itr = FileUtils.listFilesIterator(fs, path, true, FileUtils.HIDDEN_FILES_PATH_FILTER); + Assert.assertEquals(1, getFileCount(itr)); + } + + private int getFileCount(RemoteIterator<?> lfs) throws Exception { + try { + int count = 0; + while (lfs.hasNext()) { + lfs.next(); + count++; Review Comment: Can assert the expected file paths as well. Can change the util as: ``` private int assertExpectedFilePaths(RemoteIterator<? extends FileStatus> lfs, List<String> expectedPaths) throws Exception { int count = 0; while (lfs.hasNext()) { assertTrue(expectedPaths.contains(lfs.next().getPath().toString())); count++; } return count; } ``` Issue Time Tracking ------------------- Worklog Id: (was: 803534) Time Spent: 3h (was: 2h 50m) > Cleaner fails with FileNotFoundException > ---------------------------------------- > > Key: HIVE-26481 > URL: https://issues.apache.org/jira/browse/HIVE-26481 > Project: Hive > Issue Type: Bug > Reporter: KIRTI RUGE > Priority: Major > Labels: pull-request-available > Time Spent: 3h > Remaining Estimate: 0h > > The compaction fails when the Cleaner tried to remove a missing directory > from HDFS. > {code:java} > 2022-08-05 18:56:38,873 INFO org.apache.hadoop.hive.ql.txn.compactor.Cleaner: > [Cleaner-executor-thread-0]: Starting cleaning for > id:30,dbname:default,tableName:test_concur_compaction_minor,partName:null,state:�,type:MINOR,enqueueTime:0,start:0,properties:null,runAs:hive,tooManyAborts:false,hasOldAbort:false,highestWriteId:4,errorMessage:null,workerId: > null,initiatorId: null 2022-08-05 18:56:38,888 ERROR > org.apache.hadoop.hive.ql.txn.compactor.Cleaner: [Cleaner-executor-thread-0]: > Caught exception when cleaning, unable to complete cleaning of > id:30,dbname:default,tableName:test_concur_compaction_minor,partName:null,state:�,type:MINOR,enqueueTime:0,start:0,properties:null,runAs:hive,tooManyAborts:false,hasOldAbort:false,highestWriteId:4,errorMessage:null,workerId: > null,initiatorId: null java.io.FileNotFoundException: File > hdfs://ns1/warehouse/tablespace/managed/hive/test_concur_compaction_minor/.hive-staging_hive_2022-08-05_18-56-37_115_5049319600695911622-37 > does not exist. at > org.apache.hadoop.hdfs.DistributedFileSystem$DirListingIterator.<init>(DistributedFileSystem.java:1275) > at > org.apache.hadoop.hdfs.DistributedFileSystem$DirListingIterator.<init>(DistributedFileSystem.java:1249) > at > org.apache.hadoop.hdfs.DistributedFileSystem$25.doCall(DistributedFileSystem.java:1194) > at > org.apache.hadoop.hdfs.DistributedFileSystem$25.doCall(DistributedFileSystem.java:1190) > at > org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81) > at > org.apache.hadoop.hdfs.DistributedFileSystem.listLocatedStatus(DistributedFileSystem.java:1208) > at org.apache.hadoop.fs.FileSystem.listLocatedStatus(FileSystem.java:2144) > at org.apache.hadoop.fs.FileSystem$5.handleFileStat(FileSystem.java:2332) at > org.apache.hadoop.fs.FileSystem$5.hasNext(FileSystem.java:2309) at > org.apache.hadoop.hive.ql.io.AcidUtils.getHdfsDirSnapshots(AcidUtils.java:1440) > at > org.apache.hadoop.hive.ql.txn.compactor.Cleaner.removeFiles(Cleaner.java:287) > at org.apache.hadoop.hive.ql.txn.compactor.Cleaner.clean(Cleaner.java:214) at > org.apache.hadoop.hive.ql.txn.compactor.Cleaner.lambda$run$0(Cleaner.java:114) > at > org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil$ThrowingRunnable.lambda$unchecked$0(CompactorUtil.java:54) > at > java.base/java.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1736) > at > java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128) > at > java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628) > at java.base/java.lang.Thread.run(Thread.java:834){code} -- This message was sent by Atlassian Jira (v8.20.10#820010)