You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by dk...@apache.org on 2022/08/31 07:33:59 UTC

[hive] branch master updated: HIVE-26481: Cleaner fails with FileNotFoundException (Kirti Ruge, reviewed by Ayush Saxena, Denys Kuzmenko)

This is an automated email from the ASF dual-hosted git repository.

dkuzmenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new 1c310f8cebe HIVE-26481: Cleaner fails with FileNotFoundException (Kirti Ruge, reviewed by Ayush Saxena, Denys Kuzmenko)
1c310f8cebe is described below

commit 1c310f8cebe4860e759c9c48f845f4565343f2bd
Author: rkirtir <11...@users.noreply.github.com>
AuthorDate: Wed Aug 31 13:03:43 2022 +0530

    HIVE-26481: Cleaner fails with FileNotFoundException (Kirti Ruge, reviewed by Ayush Saxena, Denys Kuzmenko)
    
    Closes #3531
---
 .../org/apache/hadoop/hive/common/FileUtils.java   | 13 ++++
 .../apache/hadoop/hive/common/TestFileUtils.java   | 31 ++++++++
 .../hive/ql/txn/compactor/CompactorTestUtil.java   |  3 +-
 .../hive/ql/txn/compactor/TestCompactor.java       |  4 +-
 .../ql/txn/compactor/TestMmCompactorOnTez.java     |  4 +-
 .../org/apache/hadoop/hive/ql/io/AcidUtils.java    | 89 ++++++++++------------
 .../org/apache/hadoop/hive/ql/io/HdfsUtils.java    |  3 +-
 .../apache/hadoop/hive/ql/io/HiveInputFormat.java  |  2 +-
 .../hive/ql/parse/repl/dump/io/FileOperations.java |  3 +-
 .../org/apache/hadoop/hive/ql/TestTxnCommands.java |  8 +-
 .../hadoop/hive/ql/txn/compactor/TestWorker.java   | 14 ++--
 11 files changed, 107 insertions(+), 67 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
index e9410ec4d0a..37ff2c04dc2 100644
--- a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
@@ -62,6 +62,7 @@ import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.functional.RemoteIterators;
 import org.apache.hive.common.util.ShutdownHookManager;
 import org.apache.hive.common.util.SuppressFBWarnings;
 import org.slf4j.Logger;
@@ -1334,4 +1335,16 @@ public final class FileUtils {
       LOG.debug("Unable to delete {}", path, e);
     }
   }
+
+  public static RemoteIterator<FileStatus> listStatusIterator(FileSystem fs, Path path, PathFilter filter)
+        throws IOException {
+    return RemoteIterators.filteringRemoteIterator(fs.listStatusIterator(path),
+        status -> filter.accept(status.getPath()));
+  }
+
+  public static RemoteIterator<LocatedFileStatus> listFiles(FileSystem fs, Path path, boolean recursive, PathFilter filter)
+        throws IOException {
+    return RemoteIterators.filteringRemoteIterator(fs.listFiles(path, recursive),
+        status -> filter.accept(status.getPath()));
+  }
 }
diff --git a/common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java b/common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java
index 4f4d6046d38..2721deb7a03 100644
--- a/common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java
+++ b/common/src/test/org/apache/hadoop/hive/common/TestFileUtils.java
@@ -33,16 +33,22 @@ import java.net.URI;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.shims.HadoopShims;
 
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hive.common.util.MockFileSystem;
+import org.apache.hive.common.util.MockFileSystem.MockFile;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -281,4 +287,29 @@ public class TestFileUtils {
     relativePath = FileUtils.makeRelative(parentPath, childPath);
     assertEquals(childPath.toString(), relativePath.toString());
   }
+
+  @Test
+  public void testListStatusIterator() throws Exception {
+    MockFileSystem fs = new MockFileSystem(new HiveConf(),
+        new MockFile("mock:/tmp/.staging", 500, new byte[0]),
+        new MockFile("mock:/tmp/_dummy", 500, new byte[0]),
+        new 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);
+    assertEquals(1, assertExpectedFilePaths(it, Collections.singletonList("mock:/tmp/dummy")));
+    
+    RemoteIterator<LocatedFileStatus> itr = FileUtils.listFiles(fs, path, true, FileUtils.HIDDEN_FILES_PATH_FILTER);
+    assertEquals(1, assertExpectedFilePaths(itr, Collections.singletonList("mock:/tmp/dummy")));
+  }
+
+  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;
+  }
 }
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java
index 899d7b495dc..e157f641c56 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/CompactorTestUtil.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.metastore.txn.TxnUtils;
 import org.apache.hadoop.hive.ql.IDriver;
 import org.apache.hadoop.hive.ql.io.AcidInputFormat;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.ql.io.IOConstants;
 import org.apache.hadoop.hive.ql.io.RecordIdentifier;
 import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
@@ -111,7 +112,7 @@ class CompactorTestUtil {
       throws IOException {
     Path path = partitionName == null ? new Path(table.getSd().getLocation(), deltaName) : new Path(
         new Path(table.getSd().getLocation()), new Path(partitionName, deltaName));
-    return Arrays.stream(fs.listStatus(path, AcidUtils.hiddenFileFilter)).map(FileStatus::getPath).map(Path::getName).sorted()
+    return Arrays.stream(fs.listStatus(path,  FileUtils.HIDDEN_FILES_PATH_FILTER)).map(FileStatus::getPath).map(Path::getName).sorted()
         .collect(Collectors.toList());
   }
 
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
index e8a8ab9c49d..8206d9a7f73 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java
@@ -27,7 +27,7 @@ import static org.mockito.Mockito.doAnswer;
 import static org.mockito.Mockito.doThrow;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
-
+import org.apache.hadoop.hive.common.FileUtils;
 import java.io.File;
 import java.io.FileWriter;
 import java.io.IOException;
@@ -1714,7 +1714,7 @@ public class TestCompactor {
     verifyHasBase(table.getSd(), fs, baseDir);
 
     FileStatus[] files = fs.listStatus(new Path(table.getSd().getLocation(), baseDir),
-        AcidUtils.hiddenFileFilter);
+        FileUtils.HIDDEN_FILES_PATH_FILTER);
     Assert.assertEquals(Lists.newArrayList(files).toString(), 64, files.length);
   }
 
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMmCompactorOnTez.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMmCompactorOnTez.java
index e552f880da2..35a998248c1 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMmCompactorOnTez.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestMmCompactorOnTez.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.ql.hooks.HiveProtoLoggingHook.ExecutionMode;
 import org.apache.hadoop.hive.ql.hooks.TestHiveProtoLoggingHook;
 import org.apache.hadoop.hive.ql.hooks.proto.HiveHookEvents;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.tez.dag.history.logging.proto.ProtoMessageReader;
 import org.junit.Assert;
 import org.junit.Test;
@@ -148,12 +149,13 @@ public class TestMmCompactorOnTez extends CompactorOnTezTest {
         Collections.singletonList(newDeltaName), actualDeltasAfterComp);
     // Verify number of files in directory
     FileStatus[] files = fs.listStatus(new Path(table.getSd().getLocation(), newDeltaName),
-        AcidUtils.hiddenFileFilter);
+       FileUtils.HIDDEN_FILES_PATH_FILTER);
     Assert.assertEquals("Incorrect number of bucket files", 2, files.length);
     // Verify bucket files in delta dirs
     List<String> expectedBucketFiles = Arrays.asList("000000_0", "000001_0");
     Assert.assertEquals("Bucket names are not matching after compaction", expectedBucketFiles,
         CompactorTestUtil.getBucketFileNamesForMMTables(fs, table, null, actualDeltasAfterComp.get(0)));
+
     verifyAllContents(tableName, testDataProvider, expectedData);
     // Clean up
     testDataProvider.dropTable(tableName);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
index 79fc6743384..a9c3bb5a827 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/AcidUtils.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.ql.io;
 
 import static org.apache.hadoop.hive.common.AcidConstants.SOFT_DELETE_PATH_SUFFIX;
 import static org.apache.hadoop.hive.common.AcidConstants.SOFT_DELETE_TABLE;
+import static org.apache.hadoop.hive.common.FileUtils.HIDDEN_FILES_PATH_FILTER;
 import static org.apache.hadoop.hive.ql.exec.Utilities.COPY_KEYWORD;
 import static org.apache.hadoop.hive.ql.parse.CalcitePlanner.ASTSearcher;
 
@@ -47,6 +48,8 @@ import java.util.function.Supplier;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
+import org.apache.hadoop.hive.common.FileUtils;
+
 import com.google.common.base.Strings;
 import com.google.common.base.Preconditions;
 import com.google.common.cache.Cache;
@@ -230,14 +233,6 @@ public class AcidUtils {
   public static final Pattern ORIGINAL_PATTERN_COPY =
     Pattern.compile("[0-9]+_[0-9]+" + COPY_KEYWORD + "[0-9]+");
 
-  public static final PathFilter hiddenFileFilter = new PathFilter(){
-    @Override
-    public boolean accept(Path p){
-      String name = p.getName();
-      return !name.startsWith("_") && !name.startsWith(".");
-    }
-  };
-
   public static final PathFilter acidHiddenFileFilter = new PathFilter() {
     @Override
     public boolean accept(Path p) {
@@ -250,7 +245,7 @@ public class AcidUtils {
       if (name.startsWith(OrcAcidVersion.ACID_FORMAT)) {
         return true;
       }
-      return !name.startsWith("_") && !name.startsWith(".");
+      return HIDDEN_FILES_PATH_FILTER.accept(p);
     }
   };
 
@@ -983,7 +978,7 @@ public class AcidUtils {
     public List<HdfsFileStatusWithId> getFiles(FileSystem fs, Ref<Boolean> useFileIds) throws IOException {
       // If the list was not populated before, do it now
       if (files == null && fs != null) {
-        files = HdfsUtils.listFileStatusWithId(fs, baseDirPath, useFileIds, false, AcidUtils.hiddenFileFilter);
+        files = HdfsUtils.listFileStatusWithId(fs, baseDirPath, useFileIds, false, HIDDEN_FILES_PATH_FILTER);
       }
       return files;
     }
@@ -1338,7 +1333,7 @@ public class AcidUtils {
     FileSystem fs = fileSystem == null ? candidateDirectory.getFileSystem(conf) : fileSystem;
     AcidDirectory directory = new AcidDirectory(candidateDirectory, fs, useFileIds);
 
-    List<HdfsFileStatusWithId> childrenWithId = HdfsUtils.tryListLocatedHdfsStatus(useFileIds, fs, candidateDirectory, hiddenFileFilter);
+    List<HdfsFileStatusWithId> childrenWithId = HdfsUtils.tryListLocatedHdfsStatus(useFileIds, fs, candidateDirectory, HIDDEN_FILES_PATH_FILTER);
 
     if (childrenWithId != null) {
       for (HdfsFileStatusWithId child : childrenWithId) {
@@ -1500,25 +1495,23 @@ public class AcidUtils {
           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));
     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);
+        if (baseFileFilter.accept(fPath) ||
+                deltaFileFilter.accept(fPath) ||
+                deleteEventDeltaDirFilter.accept(fPath)) {
+          addToSnapshoot(dirToSnapshots, fPath);
+        } else {
+          if (fStatus.isDirectory()) {
+            stack.push(FileUtils.listStatusIterator(fs, fPath, acidHiddenFileFilter));
           } else {
-            if (fStatus.isDirectory()) {
-              stack.push(fs.listStatusIterator(fPath));
-            } else {
-              // Found an original file
-              HdfsDirSnapshot hdfsDirSnapshot = addToSnapshoot(dirToSnapshots, fPath.getParent());
-              hdfsDirSnapshot.addFile(fStatus);
-            }
+            // Found an original file
+            HdfsDirSnapshot hdfsDirSnapshot = addToSnapshoot(dirToSnapshots, fPath.getParent());
+            hdfsDirSnapshot.addFile(fStatus);
           }
         }
       }
@@ -1538,34 +1531,32 @@ public class AcidUtils {
   public static Map<Path, HdfsDirSnapshot> getHdfsDirSnapshots(final FileSystem fs, final Path path)
       throws IOException {
     Map<Path, HdfsDirSnapshot> dirToSnapshots = new HashMap<>();
-    RemoteIterator<LocatedFileStatus> itr = fs.listFiles(path, true);
+    RemoteIterator<LocatedFileStatus> itr = FileUtils.listFiles(fs, path, true, acidHiddenFileFilter);
     while (itr.hasNext()) {
       FileStatus fStatus = itr.next();
       Path fPath = fStatus.getPath();
-      if (acidHiddenFileFilter.accept(fPath)) {
-        if (fStatus.isDirectory() && acidTempDirFilter.accept(fPath)) {
-          addToSnapshoot(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 = addToSnapshoot(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);
-            }
+      if (fStatus.isDirectory() && acidTempDirFilter.accept(fPath)) {
+        addToSnapshoot(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 = addToSnapshoot(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);
           }
         }
       }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HdfsUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HdfsUtils.java
index 0815ad1668f..6a29ab7d370 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HdfsUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HdfsUtils.java
@@ -29,6 +29,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
 
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hive.common.util.Ref;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -223,7 +224,7 @@ public class HdfsUtils {
       return null;
     }
     if (filter == null) {
-      filter = AcidUtils.hiddenFileFilter;
+      filter = FileUtils.HIDDEN_FILES_PATH_FILTER;
     }
 
     List<HadoopShims.HdfsFileStatusWithId> childrenWithId = null;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index 403fb9ed01d..6f38d680a86 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -696,7 +696,7 @@ public class HiveInputFormat<K extends WritableComparable, V extends Writable>
     // We need to iterate to detect original directories, that are supported in MM but not ACID.
     boolean hasOriginalFiles = false, hasAcidDirs = false;
     List<Path> originalDirectories = new ArrayList<>();
-    for (FileStatus file : fs.listStatus(dir, AcidUtils.hiddenFileFilter)) {
+    for (FileStatus file : fs.listStatus(dir, FileUtils.HIDDEN_FILES_PATH_FILTER)) {
       Path currDir = file.getPath();
       Utilities.FILE_OP_LOGGER.trace("Checking {} for being an input", currDir);
       if (!file.isDirectory()) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/io/FileOperations.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/io/FileOperations.java
index a83f8c97630..9fae68667b2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/io/FileOperations.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/repl/dump/io/FileOperations.java
@@ -31,6 +31,7 @@ import javax.security.auth.login.LoginException;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.common.ValidWriteIdList;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.ReplChangeManager;
@@ -130,7 +131,7 @@ public class FileOperations {
         copyOneDataPath(validPath, destPath);
       }
       for (Path dirWithOriginals : dirsWithOriginals) {
-        FileStatus[] files = dataFileSystem.listStatus(dirWithOriginals, AcidUtils.hiddenFileFilter);
+        FileStatus[] files = dataFileSystem.listStatus(dirWithOriginals, FileUtils.HIDDEN_FILES_PATH_FILTER);
         List<Path> srcPaths = new ArrayList<>();
         for (FileStatus fileStatus : files) {
           if (fileStatus.isDirectory()) continue;
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
index 13d660f8fda..f7261c8f781 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/TestTxnCommands.java
@@ -33,7 +33,7 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
-
+import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
@@ -249,7 +249,7 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
     Assert.assertEquals(imported.toString(), "insert_only",
         imported.getParameters().get("transactional_properties"));
     Path importPath = new Path(imported.getSd().getLocation());
-    FileStatus[] stat = fs.listStatus(importPath, AcidUtils.hiddenFileFilter);
+    FileStatus[] stat = fs.listStatus(importPath, FileUtils.HIDDEN_FILES_PATH_FILTER);
     Assert.assertEquals(Arrays.toString(stat), 1, stat.length);
     assertIsDelta(stat[0]);
     List<String> allData = stringifyValues(rows1);
@@ -277,7 +277,7 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
     Assert.assertNull(imported.toString(),
         imported.getParameters().get("transactional_properties"));
     importPath = new Path(imported.getSd().getLocation());
-    stat = fs.listStatus(importPath, AcidUtils.hiddenFileFilter);
+    stat = fs.listStatus(importPath, FileUtils.HIDDEN_FILES_PATH_FILTER);
     allData = stringifyValues(rows2);
     Collections.sort(allData);
     rs = runStatementOnDriver(String.format("select a,b from %s order by a,b", importName));
@@ -746,7 +746,7 @@ public class TestTxnCommands extends TxnCommandsBaseForTests {
     queue.add(path);
     while (!queue.isEmpty()) {
       Path next = queue.pollFirst();
-      FileStatus[] stats = fs.listStatus(next, AcidUtils.hiddenFileFilter);
+      FileStatus[] stats = fs.listStatus(next, FileUtils.HIDDEN_FILES_PATH_FILTER);
       for (FileStatus stat : stats) {
         Path child = stat.getPath();
         paths.add(child.toString());
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
index 94276ccccc3..cf7e32d716c 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestWorker.java
@@ -324,7 +324,7 @@ public class TestWorker extends CompactorTest {
     for (int i = 0; i < stat.length; i++) {
       if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21, 24) + "_v0000026")) {
         sawNewDelta = true;
-        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter);
+        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
@@ -333,7 +333,7 @@ public class TestWorker extends CompactorTest {
       }
       if (stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(21, 24) + "_v0000026")) {
         sawNewDelta = true;
-        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter);
+        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
@@ -456,7 +456,7 @@ public class TestWorker extends CompactorTest {
     for (int i = 0; i < stat.length; i++) {
       if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21, 24) + "_v0000026")) {
         sawNewDelta = true;
-        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter);
+        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
@@ -465,7 +465,7 @@ public class TestWorker extends CompactorTest {
       }
       if (stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(21, 24))) {
         sawNewDelta = true;
-        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter);
+        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
@@ -508,7 +508,7 @@ public class TestWorker extends CompactorTest {
     for (int i = 0; i < stat.length; i++) {
       if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(1, 4) + "_v0000006")) {
         sawNewDelta = true;
-        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter);
+        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
@@ -517,7 +517,7 @@ public class TestWorker extends CompactorTest {
       }
       if (stat[i].getPath().getName().equals(makeDeleteDeltaDirNameCompacted(1, 4) + "_v0000006")) {
         sawNewDelta = true;
-        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter);
+        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));
@@ -840,7 +840,7 @@ public class TestWorker extends CompactorTest {
     for (int i = 0; i < stat.length; i++) {
       if (stat[i].getPath().getName().equals(makeDeltaDirNameCompacted(21, 24) + "_v0000026")) {
         sawNewDelta = true;
-        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), AcidUtils.hiddenFileFilter);
+        FileStatus[] buckets = fs.listStatus(stat[i].getPath(), FileUtils.HIDDEN_FILES_PATH_FILTER);
         Assert.assertEquals(2, buckets.length);
         Assert.assertTrue(buckets[0].getPath().getName().matches("bucket_0000[01]"));
         Assert.assertTrue(buckets[1].getPath().getName().matches("bucket_0000[01]"));