You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by GitBox <gi...@apache.org> on 2022/02/16 19:10:04 UTC

[GitHub] [hadoop] steveloughran commented on a change in pull request #3978: HADOOP-13704. Optimised getContentSummary()

steveloughran commented on a change in pull request #3978:
URL: https://github.com/apache/hadoop/pull/3978#discussion_r808346583



##########
File path: hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractContentSummaryTest.java
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.assertj.core.api.Assertions;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.FileNotFoundException;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
+public abstract class AbstractContractContentSummaryTest extends AbstractFSContractTestBase  {
+
+    @Test
+    public void testGetContentSummary() throws Throwable {
+        FileSystem fs = getFileSystem();
+
+        Path parent = path("parent");
+        Path nested = path(parent + "/a/b/c");
+        Path filePath = path(nested + "file.txt");
+
+        fs.mkdirs(parent);
+        fs.mkdirs(nested);
+        touch(getFileSystem(), filePath);
+
+        ContentSummary summary = fs.getContentSummary(parent);
+
+        Assertions.assertThat(summary.getDirectoryCount())
+                .as("Summary " + summary)
+                .isEqualTo(4);
+
+        Assertions.assertThat(summary.getFileCount())
+                .as("Summary " + summary)
+                .isEqualTo(1);
+    }
+
+    @Test
+    public void testGetContentSummaryIncorrectPath() throws Throwable {
+        FileSystem fs = getFileSystem();
+
+        Path parent = path("parent");
+        Path nested = path(parent + "/a");
+
+        fs.mkdirs(parent);
+
+        try {

Review comment:
       use lambda test utils intercept() here

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractContentSummary.java
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractContractContentSummaryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import static org.apache.hadoop.fs.contract.ContractTestUtils.touch;
+
+
+public class ITestS3AContractContentSummary extends AbstractContractContentSummaryTest {
+
+    @Test
+    public void testGetContentSummaryDir() throws Throwable {
+        describe("getContentSummary on test dir with children");
+        S3AFileSystem fs = getFileSystem();
+        Path baseDir = methodPath();
+
+        // Nested folders created separately will return as separate objects in listFiles()
+        fs.mkdirs(new Path(baseDir + "/a"));

Review comment:
       better to use
   ```
   new Path(basedir, "a");
   ```

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java
##########
@@ -204,5 +233,6 @@ S3AFileStatus probePathStatus(Path path,
     RemoteIterator<S3AFileStatus> listStatusIterator(Path path)

Review comment:
       is this method obsolete now?

##########
File path: hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractContentSummary.java
##########
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *       http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.fs.contract.s3a;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.contract.AbstractContractContentSummaryTest;
+import org.apache.hadoop.fs.contract.AbstractFSContract;
+import org.apache.hadoop.fs.s3a.S3AFileSystem;
+import org.assertj.core.api.Assertions;

Review comment:
       nit: move to their own block above this one

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java
##########
@@ -20,8 +20,10 @@
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.util.HashSet;
 import java.util.Set;
 
+import org.apache.hadoop.fs.s3a.S3ALocatedFileStatus;

Review comment:
       nit: put all org.apache. imports in their own block under the others. note, some fixup of our move off guava means many of our current files break this rule ... and moving imports around makes cherrypicking harder. so we leave those alone

##########
File path: hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/impl/GetContentSummaryOperation.java
##########
@@ -133,34 +133,63 @@ public ContentSummary execute() throws IOException {
    * @throws IOException failure
    */
   public ContentSummary getDirSummary(Path dir) throws IOException {
+
     long totalLength = 0;
     long fileCount = 0;
     long dirCount = 1;
-    final RemoteIterator<S3AFileStatus> it
-        = callbacks.listStatusIterator(dir);
+
+    RemoteIterator<S3ALocatedFileStatus> it = callbacks.listFilesIterator(dir, true);
+
+    Set<Path> dirSet = new HashSet<>();
+    Set<Path> pathsTraversed = new HashSet<>();
 
     while (it.hasNext()) {
-      final S3AFileStatus s = it.next();
-      if (s.isDirectory()) {
-        try {
-          ContentSummary c = getDirSummary(s.getPath());
-          totalLength += c.getLength();
-          fileCount += c.getFileCount();
-          dirCount += c.getDirectoryCount();
-        } catch (FileNotFoundException ignored) {
-          // path was deleted during the scan; exclude from
-          // summary.
-        }
-      } else {
-        totalLength += s.getLen();
+      S3ALocatedFileStatus fileStatus = it.next();
+      Path filePath = fileStatus.getPath();
+
+      if (fileStatus.isDirectory() && !filePath.equals(dir)) {
+        dirSet.add(filePath);
+        buildDirectorySet(dirSet, pathsTraversed, dir, filePath.getParent());
+      } else if (!fileStatus.isDirectory()) {
         fileCount += 1;
+        totalLength += fileStatus.getLen();
+        buildDirectorySet(dirSet, pathsTraversed, dir, filePath.getParent());
       }
+
     }
+
     // Add the list's IOStatistics
     iostatistics.aggregate(retrieveIOStatistics(it));
+
     return new ContentSummary.Builder().length(totalLength).
-        fileCount(fileCount).directoryCount(dirCount).
-        spaceConsumed(totalLength).build();
+            fileCount(fileCount).directoryCount(dirCount + dirSet.size()).
+            spaceConsumed(totalLength).build();
+  }
+
+  /***
+   * This method builds the set of all directories found under the base path. We need to do this because if the
+   * directory structure /a/b/c was created with a single mkdirs() call, it is stored as 1 object in S3 and the list
+   * files iterator will only return a single entry /a/b/c.
+   *
+   * We keep track of paths traversed so far to prevent duplication of work. For eg, if we had a/b/c/file-1.txt and
+   * /a/b/c/file-2.txt, we will only recurse over the complete path once and won't have to do anything for file-2.txt.
+   *
+   * @param dirSet Set of all directories found in the path
+   * @param pathsTraversed Set of all paths traversed so far
+   * @param basePath Path of directory to scan
+   * @param parentPath Parent path of the current file/directory in the iterator
+   */
+  private void buildDirectorySet(Set<Path> dirSet, Set<Path> pathsTraversed, Path basePath, Path parentPath) {
+
+    if (parentPath == null || pathsTraversed.contains(parentPath) || parentPath.equals(basePath)) {

Review comment:
       would this be faster if the parentPath equals was done before the contained() probe? or is it rare enough that it wouldn't do much/be worse




-- 
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: common-issues-unsubscribe@hadoop.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org