[ 
https://issues.apache.org/jira/browse/HIVE-27135?focusedWorklogId=852018&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-852018
 ]

ASF GitHub Bot logged work on HIVE-27135:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 21/Mar/23 13:34
            Start Date: 21/Mar/23 13:34
    Worklog Time Spent: 10m 
      Work Description: veghlaci05 commented on code in PR #4114:
URL: https://github.com/apache/hive/pull/4114#discussion_r1143382486


##########
ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java:
##########
@@ -1538,32 +1538,36 @@ private static HdfsDirSnapshot addToSnapshot(Map<Path, 
HdfsDirSnapshot> dirToSna
   public static Map<Path, HdfsDirSnapshot> getHdfsDirSnapshots(final 
FileSystem fs, final Path path)
       throws IOException {
     Map<Path, HdfsDirSnapshot> dirToSnapshots = new HashMap<>();
-    RemoteIterator<LocatedFileStatus> itr = FileUtils.listFiles(fs, path, 
true, acidHiddenFileFilter);
-    while (itr.hasNext()) {
-      FileStatus fStatus = itr.next();
-      Path fPath = fStatus.getPath();
-      if (fStatus.isDirectory() && acidTempDirFilter.accept(fPath)) {
-        addToSnapshot(dirToSnapshots, fPath);
-      } else {
-        Path parentDirPath = fPath.getParent();
-        if (acidTempDirFilter.accept(parentDirPath)) {
-          while (isChildOfDelta(parentDirPath, path)) {
-            // Some cases there are other directory layers between the delta 
and the datafiles
-            // (export-import mm table, insert with union all to mm table, 
skewed tables).
-            // But it does not matter for the AcidState, we just need the 
deltas and the data files
-            // So build the snapshot with the files inside the delta directory
-            parentDirPath = parentDirPath.getParent();
-          }
-          HdfsDirSnapshot dirSnapshot = addToSnapshot(dirToSnapshots, 
parentDirPath);
-          // We're not filtering out the metadata file and acid format file,
-          // as they represent parts of a valid snapshot
-          // We're not using the cached values downstream, but we can 
potentially optimize more in a follow-up task
-          if 
(fStatus.getPath().toString().contains(MetaDataFile.METADATA_FILE)) {
-            dirSnapshot.addMetadataFile(fStatus);
-          } else if 
(fStatus.getPath().toString().contains(OrcAcidVersion.ACID_FORMAT)) {
-            dirSnapshot.addOrcAcidFormatFile(fStatus);
-          } else {
-            dirSnapshot.addFile(fStatus);
+    Deque<RemoteIterator<LocatedFileStatus>> stack = new ArrayDeque<>();

Review Comment:
   @deniskuzZ
   I think this approach is fine for now, as @mdayakar mentioned 
`getHdfsDirSnapshotsForCleaner()` does the same. However, I would create a 
follow-up task to eliminate the workaround once HADOOP-18662 is merged.





Issue Time Tracking
-------------------

    Worklog Id:     (was: 852018)
    Time Spent: 2.5h  (was: 2h 20m)

> Cleaner fails with FileNotFoundException
> ----------------------------------------
>
>                 Key: HIVE-27135
>                 URL: https://issues.apache.org/jira/browse/HIVE-27135
>             Project: Hive
>          Issue Type: Bug
>            Reporter: Dayakar M
>            Assignee: Dayakar M
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 2.5h
>  Remaining Estimate: 0h
>
> The compaction fails when the Cleaner tried to remove a missing directory 
> from HDFS.
> {code:java}
> 2023-03-06 07:45:48,331 ERROR 
> org.apache.hadoop.hive.ql.txn.compactor.Cleaner: 
> [Cleaner-executor-thread-12]: Caught exception when cleaning, unable to 
> complete cleaning of 
> id:39762523,dbname:test,tableName:test_table,partName:null,state:,type:MINOR,enqueueTime:0,start:0,properties:null,runAs:hive,tooManyAborts:false,hasOldAbort:false,highestWriteId:989,errorMessage:null,workerId:
>  null,initiatorId: null java.io.FileNotFoundException: File 
> hdfs:/cluster/warehouse/tablespace/managed/hive/test.db/test_table/.hive-staging_hive_2023-03-06_07-45-23_120_4659605113266849995-73550
>  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.util.functional.RemoteIterators$WrappingRemoteIterator.sourceHasNext(RemoteIterators.java:432)
>     at 
> org.apache.hadoop.util.functional.RemoteIterators$FilteringRemoteIterator.fetch(RemoteIterators.java:581)
>     at 
> org.apache.hadoop.util.functional.RemoteIterators$FilteringRemoteIterator.hasNext(RemoteIterators.java:602)
>     at 
> org.apache.hadoop.hive.ql.io.AcidUtils.getHdfsDirSnapshots(AcidUtils.java:1435)
>     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.util.concurrent.CompletableFuture$AsyncRun.run(CompletableFuture.java:1640)
>     at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>     at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>     at java.lang.Thread.run(Thread.java:750){code}
> h4.  
> This issue got fixed as a part of HIVE-26481 but here its not fixed 
> completely. 
> [Here|https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java#L1541]
>  FileUtils.listFiles() API which returns a RemoteIterator<LocatedFileStatus>. 
> So while iterating over, it checks if it is a directory and recursive listing 
> then it will try to list files from that directory but if that directory is 
> removed by other thread/task then it throws FileNotFoundException. Here the 
> directory which got removed is the .staging directory which needs to be 
> excluded through by using passed filter.
>  
> So here we can use 
> _*org.apache.hadoop.hive.common.FileUtils#listStatusRecursively()*_ 
> [API|https://github.com/apache/hive/blob/master/common/src/java/org/apache/hadoop/hive/common/FileUtils.java#L372]
>  which will apply the filter before listing the files from that directory 
> which will avoid FileNotFoundException.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to