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;
     }
   ```



-- 
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

Reply via email to