[ https://issues.apache.org/jira/browse/HIVE-26481?focusedWorklogId=801882&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-801882 ]
ASF GitHub Bot logged work on HIVE-26481: ----------------------------------------- Author: ASF GitHub Bot Created on: 19/Aug/22 07:21 Start Date: 19/Aug/22 07:21 Worklog Time Spent: 10m Work Description: deniskuzZ commented on code in PR #3531: URL: https://github.com/apache/hive/pull/3531#discussion_r949881145 ########## ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java: ########## @@ -1499,29 +1499,34 @@ private static ValidTxnList getValidTxnList(Configuration conf) { public static Map<Path, HdfsDirSnapshot> getHdfsDirSnapshotsForCleaner(final FileSystem fs, final Path path) throws IOException { Map<Path, HdfsDirSnapshot> dirToSnapshots = new HashMap<>(); - Deque<RemoteIterator<FileStatus>> stack = new ArrayDeque<>(); - stack.push(fs.listStatusIterator(path)); - while (!stack.isEmpty()) { - RemoteIterator<FileStatus> itr = stack.pop(); - while (itr.hasNext()) { - FileStatus fStatus = itr.next(); - Path fPath = fStatus.getPath(); - if (acidHiddenFileFilter.accept(fPath)) { - if (baseFileFilter.accept(fPath) || - deltaFileFilter.accept(fPath) || - deleteEventDeltaDirFilter.accept(fPath)) { - addToSnapshoot(dirToSnapshots, fPath); - } else { - if (fStatus.isDirectory()) { - stack.push(fs.listStatusIterator(fPath)); + try { + Deque<RemoteIterator<FileStatus>> stack = new ArrayDeque<>(); + stack.push(fs.listStatusIterator(path)); + while (!stack.isEmpty()) { + RemoteIterator<FileStatus> itr = stack.pop(); + while (itr.hasNext()) { + FileStatus fStatus = itr.next(); + Path fPath = fStatus.getPath(); + if (acidHiddenFileFilter.accept(fPath)) { + if (baseFileFilter.accept(fPath) || + deltaFileFilter.accept(fPath) || + deleteEventDeltaDirFilter.accept(fPath)) { + addToSnapshoot(dirToSnapshots, fPath); } else { - // Found an original file - HdfsDirSnapshot hdfsDirSnapshot = addToSnapshoot(dirToSnapshots, fPath.getParent()); - hdfsDirSnapshot.addFile(fStatus); + if (fStatus.isDirectory()) { + stack.push(fs.listStatusIterator(fPath)); + } else { + // Found an original file + HdfsDirSnapshot hdfsDirSnapshot = addToSnapshoot(dirToSnapshots, fPath.getParent()); + hdfsDirSnapshot.addFile(fStatus); + } } } } } + } catch (FileNotFoundException fne) { + //ignore Review Comment: please add some comment that the current FS API doesn't provide the ability to supply a PathFilter to ignore the staging dirs, and that is why we need to catch this exception Issue Time Tracking ------------------- Worklog Id: (was: 801882) Time Spent: 40m (was: 0.5h) > 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: 40m > 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)