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 2020/12/15 07:17:40 UTC

[GitHub] [hadoop] sumangala-patki opened a new pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

sumangala-patki opened a new pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549


   Adding implementation for the HDFS method getContentSummary, which takes in a Path argument and returns details such as file/directory count and space utilized under that path.
   
   Tests added to check information in the returned ContentSummary instance for files and directories at different levels in a filesystem.


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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r615681004



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,144 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =
+      new ExecutorCompletionService<>(executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+
+  /**
+   * Processes a given path for count of subdirectories, files and total number
+   * of bytes
+   * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
+   * listStatus calls to server
+   */
+  public ContentSummaryProcessor(ListingSupport abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+          LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+              queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+      LOG.debug("Executor shutdown");
+    }
+    LOG.debug("Processed content summary of subtree under given path");
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  /**
+   * Calls listStatus on given path and populated fileStatus queue with
+   * subdirectories. Is called by new tasks to process the complete subtree
+   * under a given path
+   * @param path: Path to a file or directory
+   * @throws IOException: listStatus error
+   * @throws InterruptedException: error while inserting into queue
+   */
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();

Review comment:
       There will always be at least one task submitted when the queue is non-empty. This can be a previously submitted task that had not begun, or a new one generated in line 106 if the queue was found to be non-empty. So all items added to the queue get processed
   
   For all files, the currently executing task processes the count details (line 108), so new tasks/ threads are not required




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r555527694



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+                queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+    }
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();
+      } else {
+        processFile(fileStatus);
+      }
+    }
+  }
+
+  private void processDirectory() {
+    directoryCount.incrementAndGet();
+  }
+
+  private void processFile(FileStatus fileStatus) {
+    fileCount.incrementAndGet();
+    totalBytes.addAndGet(fileStatus.getLen());
+  }
+
+  private synchronized void conditionalSubmitTaskToExecutor() {
+    if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) {

Review comment:
       renamed




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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554932969



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,204 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private final String[] directories = {"/testFolder",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2", "/testFolder/testFolder3", "/testFolderII",

Review comment:
       keep the values in separate lines for better readability




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

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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-839710241






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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r705849850



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =
+      new ExecutorCompletionService<>(executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+
+  /**
+   * Processes a given path for count of subdirectories, files and total number
+   * of bytes
+   * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
+   * listStatus calls to server
+   */
+  public ContentSummaryProcessor(ListingSupport abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path, TracingContext tracingContext)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path, tracingContext);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+          LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+              queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+      LOG.debug("Executor shutdown");
+    }
+    LOG.debug("Processed content summary of subtree under given path");
+    ContentSummary.Builder builder = new ContentSummary.Builder()
+        .directoryCount(directoryCount.get()).fileCount(fileCount.get())
+        .length(totalBytes.get()).spaceConsumed(totalBytes.get());
+    return builder.build();
+  }
+
+  /**
+   * Calls listStatus on given path and populated fileStatus queue with
+   * subdirectories. Is called by new tasks to process the complete subtree
+   * under a given path
+   * @param path: Path to a file or directory
+   * @throws IOException: listStatus error
+   * @throws InterruptedException: error while inserting into queue
+   */
+  private void processDirectoryTree(Path path, TracingContext tracingContext)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path, tracingContext);
+
+    for (FileStatus fileStatus : fileStatuses) {

Review comment:
       Trying to confirm the advantage of processing page-wise listStatus results; would like to know your opinion. Analyzed time taken by direct liststatus call vs using listiterator (queueing subdir while iterating), but getting ambiguous results.
   
   The tests used involved creating a directory tree and calling GetContentSummary on the top folder, as the primary use of this api might be on the root of an account.
   
   Expt 1: Directory tree with 12 levels (tree height=12), where each level comprises one dir and 1-2 files.
   Expt 2: Same 12-level structure as 1, with a branch (of 2 subdir levels) around the mid-level, i.e., two subdirs at level 5, each having a subdir. All directories in the tree have ~15 files
   Expt 3: Same as expt 2, but with each dir having more than 5000 files (will result in liststatus results being fetched in multiple pages)
   
   The analysis was done for both lexicographical positions of directory with respect to files at the same level, as it determines whether the directory is fetched first. The time taken was calculated as the time between the first ListStatus REST call and the DeleteFileSystem call (post the last LS) => this will eliminate differences in file/dir creation time.
   ```
   Expt number	Dir after files		Dir before files
   1		LS (few ms)		LS
   2		LS (0.5s)		Itr (8.7s)
   3		LS (3s)			Itr (4.5s)
   ```
   
   LS(t) -> Normal direct ListStatus call was faster by t
   Itr(t) -> ListIterator was faster by t
   
   Using iterator seems beneficial for some scenarios, will go ahead with it.




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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r696894597



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,227 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_BUFFER_SIZE = 20;
+  private static final int FILES_PER_DIRECTORY = 2;
+  private static final int MAX_THREADS = 16;
+  private static final int NUM_FILES_FOR_LIST_MAX_TEST =
+      DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+private static final int NUM_CONCURRENT_CALLS = 8;
+
+  private final String[] directories = {"/testFolder",
+      "/testFolderII",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2",
+      "/testFolder/testFolder3",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7"};
+
+  private final byte[] b = new byte[TEST_BUFFER_SIZE];
+
+  public TestGetContentSummary() throws Exception {
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    createDirectoryStructure();
+    int fileCount = directories.length * FILES_PER_DIRECTORY;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    verifyContentSummary(contentSummary, directories.length, fileCount,
+        directories.length * TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testFileContentSummary() throws IOException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path filePath = new Path("/testFolder/testFile");
+    fs.create(filePath);
+    FSDataOutputStream out = fs.append(filePath);
+    out.write(b);
+    out.close();
+    ContentSummary contentSummary = fs.getContentSummary(filePath);
+    verifyContentSummary(contentSummary, 0, 1, TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testLeafDir() throws IOException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    fs.mkdirs(new Path("/testFolder/testFolder1"));
+    fs.mkdirs(new Path("/testFolder/testFolder2"));
+    Path leafDir = new Path("/testFolder/testFolder1/testFolder3");
+    fs.mkdirs(leafDir);
+    ContentSummary contentSummary = fs.getContentSummary(leafDir);
+    verifyContentSummary(contentSummary, 0, 0, 0);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesOnly()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path intermediateDir = new Path("/testFolder/testFolder1");
+    fs.mkdirs(intermediateDir);
+    populateDirWithFiles(intermediateDir, FILES_PER_DIRECTORY);
+    ContentSummary contentSummary =
+        fs.getContentSummary(intermediateDir);
+    verifyContentSummary(contentSummary, 0, FILES_PER_DIRECTORY,
+        TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesAndSubdirs()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path intermediateDir = new Path("/testFolder/testFolder1");
+    fs.mkdirs(intermediateDir);
+    populateDirWithFiles(intermediateDir, FILES_PER_DIRECTORY);
+    fs.mkdirs(new Path("/testFolder/testFolder1/testFolder3"));
+    fs.registerListener(
+        new TracingHeaderValidator(getConfiguration().getClientCorrelationId(),
+            fs.getFileSystemId(), FSOperationType.GET_CONTENT_SUMMARY, true,
+            0));
+    ContentSummary contentSummary =
+        fs.getContentSummary(intermediateDir);
+    verifyContentSummary(contentSummary, 1, FILES_PER_DIRECTORY,
+        TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testDirOverListMaxResultsItems()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path pathToListMaxDir = new Path("/testFolder/listMaxDir");
+    fs.mkdirs(pathToListMaxDir);
+    fs.mkdirs(new Path(pathToListMaxDir + "/testFolder2"));
+    populateDirWithFiles(pathToListMaxDir, NUM_FILES_FOR_LIST_MAX_TEST);
+    verifyContentSummary(
+        fs.getContentSummary(pathToListMaxDir), 1,
+        NUM_FILES_FOR_LIST_MAX_TEST, TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testInvalidPath() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    intercept(IOException.class, () -> fs.getContentSummary(new Path(
+        "/nonExistentPath")));
+  }
+
+  @Test
+  public void testConcurrentGetContentSummaryCalls()
+      throws InterruptedException, ExecutionException, IOException {
+    AzureBlobFileSystem fs = getFileSystem();
+    ExecutorService executorService = new ThreadPoolExecutor(1, MAX_THREADS, 5,
+        TimeUnit.SECONDS, new SynchronousQueue<>());
+    CompletionService<ContentSummary> completionService =
+        new ExecutorCompletionService<>(executorService);
+    createDirectoryStructure();
+    for (int i = 0; i < NUM_CONCURRENT_CALLS; i++) {
+      completionService.submit(() -> fs.getContentSummary(new Path(
+          "/testFolder")));
+    }
+    for (int i = 0; i < NUM_CONCURRENT_CALLS; i++) {
+      ContentSummary contentSummary = completionService.take().get();
+      verifyContentSummary(contentSummary, 7, 8 * FILES_PER_DIRECTORY,
+          8 * TEST_BUFFER_SIZE);
+    }
+    executorService.shutdown();
+  }
+
+  private void verifyContentSummary(ContentSummary contentSummary,
+      long expectedDirectoryCount, long expectedFileCount, long expectedByteCount) {
+    Assertions.assertThat(contentSummary.getDirectoryCount())
+        .describedAs("Incorrect directory count").isEqualTo(expectedDirectoryCount);
+    Assertions.assertThat(contentSummary.getFileCount())
+        .describedAs("Incorrect file count").isEqualTo(expectedFileCount);
+    Assertions.assertThat(contentSummary.getLength())
+        .describedAs("Incorrect length").isEqualTo(expectedByteCount);
+    Assertions.assertThat(contentSummary.getSpaceConsumed())
+        .describedAs("Incorrect value of space consumed").isEqualTo(expectedByteCount);
+  }
+
+  private void createDirectoryStructure()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    for (String directory : directories) {
+      Path dirPath = new Path(directory);
+      fs.mkdirs(dirPath);
+      populateDirWithFiles(dirPath, FILES_PER_DIRECTORY);
+    }
+  }
+
+  private void populateDirWithFiles(Path directory, int numFiles)
+      throws ExecutionException, InterruptedException, IOException {
+    final List<Future<Void>> tasks = new ArrayList<>();
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < numFiles; i++) {
+      final Path fileName = new Path(directory + "/test" + i);

Review comment:
       done




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


[GitHub] [hadoop] sumangala-patki commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-783557071


   TEST RESULTS
   
   HNS Account Location: East US 2
   NonHNS Account Location: East US 2, Central US
   
   ```
   HNS OAuth
   
   [INFO] Tests run: 101, Failures: 0, Errors: 0, Skipped: 0
   [WARNING] Tests run: 445, Failures: 0, Errors: 0, Skipped: 59
   [WARNING] Tests run: 257, Failures: 0, Errors: 0, Skipped: 48
   
   HNS SharedKey
   
   [INFO] Tests run: 101, Failures: 0, Errors: 0, Skipped: 0
   [WARNING] Tests run: 495, Failures: 0, Errors: 0, Skipped: 24
   [WARNING] Tests run: 257, Failures: 0, Errors: 0, Skipped: 40
   
   Non-HNS SharedKey
   
   [INFO] Tests run: 101, Failures: 0, Errors: 0, Skipped: 0
   [WARNING] Tests run: 502, Failures: 0, Errors: 0, Skipped: 249
   [WARNING] Tests run: 257, Failures: 0, Errors: 0, Skipped: 40
   ```


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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548359229



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +366,20 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    org.apache.hadoop.fs.azurebfs.utils.ContentSummary contentSummary = null;

Review comment:
       Do we need to specify the full package name? avoid if possible.




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r705849850



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =
+      new ExecutorCompletionService<>(executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+
+  /**
+   * Processes a given path for count of subdirectories, files and total number
+   * of bytes
+   * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
+   * listStatus calls to server
+   */
+  public ContentSummaryProcessor(ListingSupport abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path, TracingContext tracingContext)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path, tracingContext);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+          LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+              queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+      LOG.debug("Executor shutdown");
+    }
+    LOG.debug("Processed content summary of subtree under given path");
+    ContentSummary.Builder builder = new ContentSummary.Builder()
+        .directoryCount(directoryCount.get()).fileCount(fileCount.get())
+        .length(totalBytes.get()).spaceConsumed(totalBytes.get());
+    return builder.build();
+  }
+
+  /**
+   * Calls listStatus on given path and populated fileStatus queue with
+   * subdirectories. Is called by new tasks to process the complete subtree
+   * under a given path
+   * @param path: Path to a file or directory
+   * @throws IOException: listStatus error
+   * @throws InterruptedException: error while inserting into queue
+   */
+  private void processDirectoryTree(Path path, TracingContext tracingContext)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path, tracingContext);
+
+    for (FileStatus fileStatus : fileStatuses) {

Review comment:
       Trying to confirm the advantage of processing page-wise listStatus results; would like to know your opinion. Analyzed time taken by direct liststatus call vs using listiterator (queueing subdir while iterating), but getting ambiguous results.
   
   The tests used involved creating a directory tree and calling GetContentSummary on the top folder, as the primary use of this api might be on the root of an account.
   
   Expt 1: Directory tree with 12 levels (tree height=12), where each level comprises one dir and 1-2 files.
   Expt 2: Same 12-level structure as 1, with a branch (of 2 subdir levels) around the mid-level, i.e., two subdirs at level 5, each having a subdir. All directories in the tree have ~15 files
   Expt 3: Same as expt 2, but with each dir having more than 5000 files (will result in liststatus results being fetched in multiple pages)
   
   The analysis was done for both lexicographical positions of directory with respect to files at the same level, as it determines whether the directory is fetched first. The time taken was calculated as the time between the first ListStatus REST call and the DeleteFileSystem call (post the last LS) => this will eliminate differences in file/dir creation time.
   ```
   Expt number	Dir after files		Dir before files
   1		LS (few ms)		LS
   2		LS (0.5s)		Itr (8.7s)
   3		LS (3s)			Itr (4.5s)
   ```
   
   LS(t) -> Normal direct ListStatus call was faster by t
   Itr(t) -> ListIterator was faster by t
   
   Using iterator seems beneficial for some scenarios, should we go ahead with it?




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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548437967



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +366,20 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    org.apache.hadoop.fs.azurebfs.utils.ContentSummary contentSummary = null;

Review comment:
       renamed to ABFSContentSummary




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-874062488


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 42s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 52s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   0m 59s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 15s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 35s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  1s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m  4s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  6s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 15s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/31/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 5e510ce929d1 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / be0e94cb8d74789efc490dadbbdcaeac18e07bab |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/31/testReport/ |
   | Max. process+thread count | 687 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/31/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r552455439



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.Listener;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor implements AutoCloseable {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Void>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+  private Listener listener = null;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0
+          || ((ThreadPoolExecutor) executorService).getActiveCount() > 0) {
+        numTasks.decrementAndGet();
+        completionService.take().get();
+        if (listener != null) {
+          listener.checkInterrupt();
+        }
+      }
+    } catch (ExecutionException e) {
+      LOG.debug(e.getMessage());
+      throw new IOException(e);
+    } finally {
+      executorService.shutdown();
+      if (listener != null) {
+        listener.checkShutdown(((ThreadPoolExecutor)executorService).getActiveCount());
+      }
+    }
+
+//    close();
+    executorService.shutdownNow();
+    if (listener != null) {
+      // statement reachable only when no exceptions thrown by threads
+      listener.checkAllTasksComplete(numTasks,
+          ((ThreadPoolExecutor)executorService).getActiveCount());
+    }
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+    if (listener != null) {
+      synchronized (this) {
+        listener.verifyThreadCount(numTasks,
+            (ThreadPoolExecutor) executorService, NUM_THREADS);
+      }
+    }
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) {
+            numTasks.incrementAndGet();
+            Future<Void> future = completionService.submit(() -> {
+              FileStatus fileStatus1;
+              while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS))
+                  != null) {
+                processDirectoryTree(fileStatus1.getPath());
+              }
+              if (listener != null) {
+                synchronized (this) {
+                  if (!listener.isInterrupted() && listener.shouldInterrupt()) {
+                    listener.setInterrupted();
+                    throw new InterruptedException();
+                  }
+                }
+              }
+              return null;
+            });
+            futures.add(future);

Review comment:
       remove futures list




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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548359350



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +366,20 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {

Review comment:
       f doesn't look a good name, could you rename it to path or something meaningful.




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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554932129



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,204 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private final String[] directories = {"/testFolder",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2", "/testFolder/testFolder3", "/testFolderII",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7",
+      "/testFolder/testFolder3/testFolder6/leafDir",
+      "/testFolderII/listMaxDir",
+      "/testFolderII/listMaxDir/" + DEFAULT_AZURE_LIST_MAX_RESULTS/2 + "_mid_folder"};
+
+  private final Path pathToFile = new Path("/testFolder/test1");
+  private final Path pathToListMaxDir = new Path("/testFolderII/listMaxDir");
+  private final Path pathToLeafDir =
+      new Path("/testFolder/testFolder3/testFolder6/leafDir");
+  private final Path pathToIntermediateDirWithFilesOnly = new Path(
+        "/testFolder/testFolder2/testFolder5");
+  private final Path pathToIntermediateDirWithFilesAndSubdirs = new Path(
+        "/testFolder/testFolder3");
+  private final String[] dirsWithNonEmptyFiles = {"/testFolder", "/testFolder/testFolder1",
+      "/testFolder/testFolder2/testFolder5", "/testFolder/testFolder3"};
+
+  private final AzureBlobFileSystem fs = createFileSystem();
+  private final int testBufferSize = 20;
+  private final int filesPerDirectory = 2;
+  private final int numFilesForListMaxTest = DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+  private final byte[] b = new byte[testBufferSize];
+  private final int maxThreads = 16;
+
+  public TestGetContentSummary() throws Exception {
+    createDirectoryStructure();
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException {
+    int fileCount =
+        (directories.length - 2) * filesPerDirectory + numFilesForListMaxTest;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    checkContentSummary(contentSummary, directories.length, fileCount,

Review comment:
       nit: verifyContentSummary() ?




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-756385141


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   1m  7s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 45s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 16s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 17s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/11/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 19 new + 2 unchanged - 0 fixed = 21 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  6s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 59s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 29s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  77m 49s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/11/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 0806b2390c5b 4.15.0-65-generic #74-Ubuntu SMP Tue Sep 17 17:06:04 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / b612c310c26 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/11/testReport/ |
   | Max. process+thread count | 709 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/11/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548360022



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ContentSummary;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);

Review comment:
       Why do we need to use AtomicLong? AtomicLong is costly and intended to use for multithreaded code.




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-917866130


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 40s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 2 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 26s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 28s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  8s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 33s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 53s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |   0m 30s | [/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/35/artifact/out/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 25 unchanged - 1 fixed = 26 total (was 26)  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |   0m 28s | [/results-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/35/artifact/out/results-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 19 unchanged - 1 fixed = 20 total (was 20)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  9s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m  3s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 56s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 54s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/35/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 5416dc892d85 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / b64b492596f56f467867801e1c850a10948d00a2 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/35/testReport/ |
   | Max. process+thread count | 638 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/35/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r641271631



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,144 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =
+      new ExecutorCompletionService<>(executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+
+  /**
+   * Processes a given path for count of subdirectories, files and total number
+   * of bytes
+   * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
+   * listStatus calls to server
+   */
+  public ContentSummaryProcessor(ListingSupport abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+          LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+              queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+      LOG.debug("Executor shutdown");
+    }
+    LOG.debug("Processed content summary of subtree under given path");
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  /**
+   * Calls listStatus on given path and populated fileStatus queue with
+   * subdirectories. Is called by new tasks to process the complete subtree
+   * under a given path
+   * @param path: Path to a file or directory
+   * @throws IOException: listStatus error
+   * @throws InterruptedException: error while inserting into queue
+   */
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();

Review comment:
       Thanks for the link. On tests against the ForkJoinPool implementation of GetContentSummary, current mechanism is still better in terms of perf so planning to retain it




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

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


[GitHub] [hadoop] vinaysbadami commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
vinaysbadami commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r553852099



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,107 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        completionService.take().get();
+        numTasks.decrementAndGet();
+      }
+    } finally {
+      numTasks.decrementAndGet();
+      executorService.shutdown();
+    }
+    executorService.awaitTermination(1, TimeUnit.SECONDS);
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);

Review comment:
       what if this throws?




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-756614368


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 33s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 40s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 25s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 16s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/12/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 7 new + 2 unchanged - 0 fixed = 9 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 28s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  1s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 32s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  77m 40s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/12/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 8de313676685 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 1b1791075a8 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/12/testReport/ |
   | Max. process+thread count | 709 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/12/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r555527292



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +367,22 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      ABFSContentSummary contentSummary =
+          (new ContentSummaryProcessor(abfsStore)).getContentSummary(path);
+      return new Builder().length(contentSummary.getLength())
+          .directoryCount(contentSummary.getDirectoryCount())
+          .fileCount(contentSummary.getFileCount())
+          .spaceConsumed(contentSummary.getSpaceConsumed()).build();
+    } catch (InterruptedException | ExecutionException e) {

Review comment:
       done




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r696890141



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -433,6 +435,31 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  /**
+   * Returns a ContentSummary instance containing the count of directories,
+   * files and total number of bytes under a given path
+   * @param path The given path
+   * @return ContentSummary
+   * @throws IOException if an error is encountered during listStatus calls
+   * or if there is any issue with the thread pool used while processing
+   */
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      TracingContext tracingContext = new TracingContext(clientCorrelationId,
+          fileSystemId, FSOperationType.GET_CONTENT_SUMMARY, true,
+          tracingHeaderFormat, listener);
+      return (new ContentSummaryProcessor(abfsStore)).getContentSummary(path,
+          tracingContext);
+    } catch (InterruptedException e) {
+      LOG.debug("Thread interrupted");
+      throw new IOException(e);
+    } catch(ExecutionException ex) {
+      LOG.debug("GetContentSummary failed with error: {}", ex.getMessage());
+      throw new IOException(ex);

Review comment:
       done




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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548951898



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,123 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Object>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int NUM_THREADS = 16;
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (!queue.isEmpty() || numTasks.get() > 0) {
+      for (Future<Object> future : futures) {
+        try {
+          future.get(10, TimeUnit.MILLISECONDS);

Review comment:
       This is not really needed for termination since the while condition is sufficient.
   Findbugs requires us to use returned values from functions, so one way out is to collect the future objects returned and wait on them at the end of main process. In our case, the callable returns null on successful completion of task. Had we used a runnable task (void return type), this would not be necessary. But runnable does not allow us to throw checked exceptions, so we have to use callable (and the returned future).
   The other option is to SuppressWarnings by findbugs.




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-756772717


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 31s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 31s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 38s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  0s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/13/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 2 new + 2 unchanged - 0 fixed = 4 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  0s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 21s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  77m  9s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/13/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 53bc5fea06a5 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 41767599093 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/13/testReport/ |
   | Max. process+thread count | 689 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/13/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-774694202


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 31s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  5s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 37s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  0s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 15s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  12m 41s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  1s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 55s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 31s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 37s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/20/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 764e826c8b6f 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 19ae0faacc5 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/20/testReport/ |
   | Max. process+thread count | 624 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/20/console |
   | versions | git=2.25.1 maven=3.6.3 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] surendralilhore commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
surendralilhore commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r614661413



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,144 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =
+      new ExecutorCompletionService<>(executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+
+  /**
+   * Processes a given path for count of subdirectories, files and total number
+   * of bytes
+   * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
+   * listStatus calls to server
+   */
+  public ContentSummaryProcessor(ListingSupport abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+          LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+              queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+      LOG.debug("Executor shutdown");
+    }
+    LOG.debug("Processed content summary of subtree under given path");
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  /**
+   * Calls listStatus on given path and populated fileStatus queue with
+   * subdirectories. Is called by new tasks to process the complete subtree
+   * under a given path
+   * @param path: Path to a file or directory
+   * @throws IOException: listStatus error
+   * @throws InterruptedException: error while inserting into queue
+   */
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();

Review comment:
       What if still some item queued and there is no directory to traverse, who will process remaining queued item ?




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548438398



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ContentSummary;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  //cached thread pool with custom max threads to avoid overloading

Review comment:
       was not required, removed




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-754446269


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 37s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 2 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | -1 :x: |  mvninstall  |  32m 50s | [/branch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/branch-mvninstall-root.txt) |  root in trunk failed.  |
   | -1 :x: |  compile  |   0m 25s | [/branch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/branch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  compile  |   0m 31s | [/branch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/branch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in trunk failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   0m 27s | [/buildtool-branch-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/buildtool-branch-checkstyle-hadoop-tools_hadoop-azure.txt) |  The patch fails to run checkstyle in hadoop-azure  |
   | -1 :x: |  mvnsite  |   0m 33s | [/branch-mvnsite-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/branch-mvnsite-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in trunk failed.  |
   | +1 :green_heart: |  shadedclient  |   1m 39s |  |  branch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 29s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  javadoc  |   0m 29s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in trunk failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | +0 :ok: |  spotbugs  |   3m 11s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | -1 :x: |  findbugs  |   0m 29s | [/branch-findbugs-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/branch-findbugs-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in trunk failed.  |
   | -0 :warning: |  patch  |   3m 42s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 24s | [/patch-mvninstall-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-mvninstall-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  compile  |   0m 23s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  javac  |   0m 23s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  compile  |   0m 22s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -1 :x: |  javac  |   0m 22s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   0m 21s | [/buildtool-patch-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/buildtool-patch-checkstyle-hadoop-tools_hadoop-azure.txt) |  The patch fails to run checkstyle in hadoop-azure  |
   | -1 :x: |  mvnsite  |   0m 23s | [/patch-mvnsite-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-mvnsite-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |   0m 21s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 23s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  javadoc  |   0m 23s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -1 :x: |  findbugs  |   0m 24s | [/patch-findbugs-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-findbugs-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 23s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | +0 :ok: |  asflicense  |   0m 24s |  |  ASF License check generated no output?  |
   |  |   |  45m 14s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux b517924a97f8 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 2b4febcf576 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/testReport/ |
   | Max. process+thread count | 94 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/10/console |
   | versions | git=2.17.1 maven=3.6.0 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r555528200



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,204 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private final String[] directories = {"/testFolder",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2", "/testFolder/testFolder3", "/testFolderII",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7",
+      "/testFolder/testFolder3/testFolder6/leafDir",
+      "/testFolderII/listMaxDir",
+      "/testFolderII/listMaxDir/" + DEFAULT_AZURE_LIST_MAX_RESULTS/2 + "_mid_folder"};
+
+  private final Path pathToFile = new Path("/testFolder/test1");
+  private final Path pathToListMaxDir = new Path("/testFolderII/listMaxDir");
+  private final Path pathToLeafDir =
+      new Path("/testFolder/testFolder3/testFolder6/leafDir");
+  private final Path pathToIntermediateDirWithFilesOnly = new Path(
+        "/testFolder/testFolder2/testFolder5");
+  private final Path pathToIntermediateDirWithFilesAndSubdirs = new Path(
+        "/testFolder/testFolder3");
+  private final String[] dirsWithNonEmptyFiles = {"/testFolder", "/testFolder/testFolder1",
+      "/testFolder/testFolder2/testFolder5", "/testFolder/testFolder3"};
+
+  private final AzureBlobFileSystem fs = createFileSystem();
+  private final int testBufferSize = 20;
+  private final int filesPerDirectory = 2;
+  private final int numFilesForListMaxTest = DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+  private final byte[] b = new byte[testBufferSize];
+  private final int maxThreads = 16;
+
+  public TestGetContentSummary() throws Exception {
+    createDirectoryStructure();
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException {
+    int fileCount =
+        (directories.length - 2) * filesPerDirectory + numFilesForListMaxTest;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    checkContentSummary(contentSummary, directories.length, fileCount,
+        dirsWithNonEmptyFiles.length * filesPerDirectory * testBufferSize);
+  }
+
+  @Test
+  public void testFileContentSummary() throws IOException {
+    ContentSummary contentSummary = fs.getContentSummary(pathToFile);
+    checkContentSummary(contentSummary, 0, 1, testBufferSize);
+  }
+
+  @Test
+  public void testLeafDir() throws IOException {
+    ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir);
+    checkContentSummary(contentSummary, 0, 0, 0);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesOnly() throws IOException {
+    ContentSummary contentSummary =
+        fs.getContentSummary(pathToIntermediateDirWithFilesOnly);
+    checkContentSummary(contentSummary, 0, filesPerDirectory,
+        testBufferSize * filesPerDirectory);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesAndSubdirs() throws IOException {
+    ContentSummary contentSummary =
+        fs.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs);
+    checkContentSummary(contentSummary, 3, 3 * filesPerDirectory,
+        testBufferSize * filesPerDirectory);
+  }
+
+  @Test
+  public void testDirOverListMaxResultsItems()
+      throws IOException {
+    checkContentSummary(
+        fs.getContentSummary(pathToListMaxDir), 1,
+        numFilesForListMaxTest + filesPerDirectory, 0);
+  }
+
+  @Test
+  public void testInvalidPath() throws Exception {
+    intercept(IOException.class, () -> fs.getContentSummary(new Path(
+        "/nonExistentPath")));
+  }
+
+  @Test
+  public void testConcurrentGetContentSummaryCalls()
+          throws InterruptedException, ExecutionException {
+    ExecutorService executorService = new ThreadPoolExecutor(1,
+            maxThreads, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+    ArrayList<Future<ContentSummary>> futures = new ArrayList<>();
+    for (String directory : directories) {
+      Future<ContentSummary> future = executorService.submit(
+              () -> fs.getContentSummary(new Path(directory)));
+      futures.add(future);
+    }
+    int[][] dirCS = {{8, 8 * filesPerDirectory, 8 * testBufferSize}, {0, filesPerDirectory, 2 * testBufferSize},
+            {2, 3 * filesPerDirectory, 2 * testBufferSize}, {3, 3 * filesPerDirectory, 2 * testBufferSize},
+            {2, numFilesForListMaxTest + 2 * filesPerDirectory, 0}, {0, filesPerDirectory, 0},
+            {0, filesPerDirectory, filesPerDirectory * testBufferSize}, {1, filesPerDirectory, 0},
+            {0, filesPerDirectory, 0}, {0, 0, 0}, {1, numFilesForListMaxTest + 2, 0}, {0, filesPerDirectory, 0}};
+    executorService.shutdown();
+    for (int i=0; i<directories.length; i++) {
+      ContentSummary contentSummary = futures.get(i).get();
+      checkContentSummary(contentSummary, dirCS[i][0], dirCS[i][1], dirCS[i][2]);
+    }
+  }
+
+  private void checkContentSummary(ContentSummary contentSummary,

Review comment:
       done




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

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


[GitHub] [hadoop] surendralilhore commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
surendralilhore commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r614674625



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -393,6 +397,32 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  /**
+   * Returns a ContentSummary instance containing the count of directories,
+   * files and total number of bytes under a given path
+   * @param path The given path
+   * @return ContentSummary
+   * @throws IOException if an error is encountered during listStatus calls
+   * or if there is any issue with the thread pool used while processing
+   */
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      ABFSContentSummary contentSummary =
+          (new ContentSummaryProcessor(abfsStore)).getContentSummary(path);

Review comment:
       Can we return directly ContentSummary object, why we need to create ABFSContentSummary class ?




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r555527894



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+                queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+    }
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();
+      } else {
+        processFile(fileStatus);
+      }
+    }
+  }
+
+  private void processDirectory() {
+    directoryCount.incrementAndGet();
+  }
+
+  private void processFile(FileStatus fileStatus) {
+    fileCount.incrementAndGet();
+    totalBytes.addAndGet(fileStatus.getLen());
+  }
+
+  private synchronized void conditionalSubmitTaskToExecutor() {
+    if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) {
+      numTasks.incrementAndGet();

Review comment:
       Can't find any op without returning, will have to stick to this

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,204 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private final String[] directories = {"/testFolder",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2", "/testFolder/testFolder3", "/testFolderII",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7",
+      "/testFolder/testFolder3/testFolder6/leafDir",
+      "/testFolderII/listMaxDir",
+      "/testFolderII/listMaxDir/" + DEFAULT_AZURE_LIST_MAX_RESULTS/2 + "_mid_folder"};
+
+  private final Path pathToFile = new Path("/testFolder/test1");
+  private final Path pathToListMaxDir = new Path("/testFolderII/listMaxDir");
+  private final Path pathToLeafDir =
+      new Path("/testFolder/testFolder3/testFolder6/leafDir");
+  private final Path pathToIntermediateDirWithFilesOnly = new Path(
+        "/testFolder/testFolder2/testFolder5");
+  private final Path pathToIntermediateDirWithFilesAndSubdirs = new Path(
+        "/testFolder/testFolder3");
+  private final String[] dirsWithNonEmptyFiles = {"/testFolder", "/testFolder/testFolder1",
+      "/testFolder/testFolder2/testFolder5", "/testFolder/testFolder3"};
+
+  private final AzureBlobFileSystem fs = createFileSystem();
+  private final int testBufferSize = 20;
+  private final int filesPerDirectory = 2;
+  private final int numFilesForListMaxTest = DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+  private final byte[] b = new byte[testBufferSize];
+  private final int maxThreads = 16;
+
+  public TestGetContentSummary() throws Exception {
+    createDirectoryStructure();
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException {
+    int fileCount =
+        (directories.length - 2) * filesPerDirectory + numFilesForListMaxTest;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    checkContentSummary(contentSummary, directories.length, fileCount,

Review comment:
       done

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,204 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private final String[] directories = {"/testFolder",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2", "/testFolder/testFolder3", "/testFolderII",

Review comment:
       done




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-757508813


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 33s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 44s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 10s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  2s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 19s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 53s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 21s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  77m 38s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/15/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 0842c82837e5 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / ec22850dbef |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/15/testReport/ |
   | Max. process+thread count | 685 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/15/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548359550



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +366,20 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    org.apache.hadoop.fs.azurebfs.utils.ContentSummary contentSummary = null;
+    try {
+      contentSummary = (new ContentSummaryProcessor(abfsStore)).getContentSummary(f);
+    } catch (InterruptedException e) {
+      e.printStackTrace();

Review comment:
       Plase remove printstacktrace. Log the exception and rethrow as an IOException.




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-906692041


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 41s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  30m 52s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 41s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  1s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 13s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 32s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 27s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/32/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 2 new + 1 unchanged - 1 fixed = 3 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  1s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  13m 56s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  4s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  72m 18s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/32/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux fe4356e628a2 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / c9d65aa0c4d5faed01e6f62abbc803739a06ae80 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/32/testReport/ |
   | Max. process+thread count | 548 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/32/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-874062488


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 42s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 52s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 26s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   0m 59s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 15s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 35s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  1s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m  4s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  6s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 15s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/31/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 5e510ce929d1 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / be0e94cb8d74789efc490dadbbdcaeac18e07bab |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/31/testReport/ |
   | Max. process+thread count | 687 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/31/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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


[GitHub] [hadoop] steveloughran commented on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
steveloughran commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-1074248757


   afraid my manifest committer changes have broken this. can you rebase and we can get this in.


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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-745500153


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 47s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 52s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 15s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 58s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/2/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 13 new + 2 unchanged - 0 fixed = 15 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 13s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 20s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 30s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  80m 21s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 37c2dbc90d96 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / df35c7f5199 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/2/testReport/ |
   | Max. process+thread count | 738 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/2/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r552443912



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.Listener;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor implements AutoCloseable {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Void>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+  private Listener listener = null;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0
+          || ((ThreadPoolExecutor) executorService).getActiveCount() > 0) {
+        numTasks.decrementAndGet();
+        completionService.take().get();
+        if (listener != null) {
+          listener.checkInterrupt();
+        }
+      }
+    } catch (ExecutionException e) {

Review comment:
       dont catch here




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-811247660


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 41s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 37s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  1s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  1s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 19s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 18s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 58s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  13m 54s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 51s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m  9s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/26/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 58b146d8d32b 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 9b2723b6c347ebc9450e36e1137464a7b1a704bd |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/26/testReport/ |
   | Max. process+thread count | 536 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/26/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554923224



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,

Review comment:
       Use constants




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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548361229



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java
##########
@@ -0,0 +1,168 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import org.apache.hadoop.fs.ContentSummary;

Review comment:
       import order




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r615673049



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -393,6 +397,32 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  /**
+   * Returns a ContentSummary instance containing the count of directories,
+   * files and total number of bytes under a given path
+   * @param path The given path
+   * @return ContentSummary
+   * @throws IOException if an error is encountered during listStatus calls
+   * or if there is any issue with the thread pool used while processing
+   */
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      ABFSContentSummary contentSummary =
+          (new ContentSummaryProcessor(abfsStore)).getContentSummary(path);
+      return new Builder().length(contentSummary.getLength())
+          .directoryCount(contentSummary.getDirectoryCount())
+          .fileCount(contentSummary.getFileCount())
+          .spaceConsumed(contentSummary.getSpaceConsumed()).build();
+    } catch (InterruptedException e) {
+      LOG.debug("Thread interrupted");
+      throw new IOException(e);
+    } catch(ExecutionException ex) {
+      LOG.debug(ex.getCause().getMessage());

Review comment:
       done




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548494660



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final AtomicInteger NUM_TASKS = new AtomicInteger(0);

Review comment:
       corrected

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final AtomicInteger NUM_TASKS = new AtomicInteger(0);
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while(!queue.isEmpty() || NUM_TASKS.get() > 0) {
+      Thread.sleep(10);
+    }
+    executorService.shutdown();
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);

Review comment:
       resolved




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r696893602



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,227 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {

Review comment:
       renamed, will send requests to store




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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-1008578613


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  12m 47s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 2 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  2s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 42s |  |  trunk passed with JDK Ubuntu-11.0.13+8-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  checkstyle  |   0m 29s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 43s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.13+8-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   1m  6s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  20m 33s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  20m 53s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  the patch passed with JDK Ubuntu-11.0.13+8-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 19s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Ubuntu-11.0.13+8-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07  |
   | +1 :green_heart: |  spotbugs  |   1m  4s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  20m 13s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  4s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 35s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  99m 10s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/2/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 2ce3d82ac628 4.15.0-112-generic #113-Ubuntu SMP Thu Jul 9 23:41:39 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 137627d019e7afcf8853af57fc01ed2e166a22d2 |
   | Default Java | Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.13+8-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_312-8u312-b07-0ubuntu1~20.04-b07 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/2/testReport/ |
   | Max. process+thread count | 545 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/2/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-750867784


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 31s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 55s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 35s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 16s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  3s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  findbugs  |   1m  0s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/7/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 21s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  78m  1s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Exceptional return value of java.util.concurrent.ExecutorService.submit(Callable) ignored in org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor.processDirectoryTree(Path)  At ContentSummaryProcessor.java:ignored in org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor.processDirectoryTree(Path)  At ContentSummaryProcessor.java:[line 79] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/7/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 750636e9c8d8 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 16a20503cac |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/7/testReport/ |
   | Max. process+thread count | 630 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/7/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-749722078


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 48s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 21s | [/branch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/branch-mvninstall-root.txt) |  root in trunk failed.  |
   | -1 :x: |  compile  |   0m 23s | [/branch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/branch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  compile  |   0m 23s | [/branch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/branch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in trunk failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   0m 21s | [/buildtool-branch-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/buildtool-branch-checkstyle-hadoop-tools_hadoop-azure.txt) |  The patch fails to run checkstyle in hadoop-azure  |
   | -1 :x: |  mvnsite  |   0m 22s | [/branch-mvnsite-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/branch-mvnsite-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in trunk failed.  |
   | +1 :green_heart: |  shadedclient  |   1m  9s |  |  branch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 23s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  javadoc  |   0m 23s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in trunk failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | +0 :ok: |  spotbugs  |   2m 20s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | -1 :x: |  findbugs  |   0m 25s | [/branch-findbugs-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/branch-findbugs-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in trunk failed.  |
   | -0 :warning: |  patch  |   2m 44s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   0m 24s | [/patch-mvninstall-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/patch-mvninstall-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | -1 :x: |  compile  |   0m 26s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  javac  |   0m 26s | [/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  compile  |   0m 22s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -1 :x: |  javac  |   0m 22s | [/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/patch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -0 :warning: |  checkstyle  |   3m 42s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 5 new + 0 unchanged - 0 fixed = 5 total (was 0)  |
   | +1 :green_heart: |  mvnsite  |   0m 50s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | -1 :x: |  shadedclient  |  14m 12s |  |  patch has errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 25s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in the patch failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  javadoc  |   0m 24s | [/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/patch-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in the patch failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | -1 :x: |  findbugs  |   0m 25s | [/patch-findbugs-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/patch-findbugs-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   |||| _ Other Tests _ |
   | -1 :x: |  unit  |   0m 25s | [/patch-unit-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/patch-unit-hadoop-tools_hadoop-azure.txt) |  hadoop-azure in the patch failed.  |
   | +0 :ok: |  asflicense  |   0m 25s |  |  ASF License check generated no output?  |
   |  |   |  29m 48s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 9477d3dff3a3 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 16a20503cac |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/testReport/ |
   | Max. process+thread count | 88 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/5/console |
   | versions | git=2.17.1 maven=3.6.0 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554586230



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,107 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        completionService.take().get();
+        numTasks.decrementAndGet();
+      }
+    } finally {
+      numTasks.decrementAndGet();
+      executorService.shutdown();
+    }
+    executorService.awaitTermination(1, TimeUnit.SECONDS);
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);

Review comment:
       caught in the azureblobfilesystem method




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

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


[GitHub] [hadoop] sumangala-patki commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-751183070


   TEST RESULTS
   
   HNS Account Location: East US 2
   NonHNS Account Location: East US 2, Central US
   
   ```
   HNS OAuth
   
   [INFO] Tests run: 97, Failures: 0, Errors: 0, Skipped: 0
   [WARNING] Tests run: 462, Failures: 0, Errors: 0, Skipped: 68
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 24
   
   HNS SharedKey
   
   [INFO] Tests run: 97, Failures: 0, Errors: 0, Skipped: 0
   [WARNING] Tests run: 462, Failures: 0, Errors: 0, Skipped: 24
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 16
   
   Non-HNS SharedKey
   
   [INFO] Tests run: 97, Failures: 0, Errors: 0, Skipped: 0
   [WARNING] Tests run: 462, Failures: 0, Errors: 0, Skipped: 249
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 16
   ```


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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548360419



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ContentSummary;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  //cached thread pool with custom max threads to avoid overloading

Review comment:
       Could you recheck if this comment is correct?




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r552442074



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.Listener;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor implements AutoCloseable {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Void>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+  private Listener listener = null;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0
+          || ((ThreadPoolExecutor) executorService).getActiveCount() > 0) {
+        numTasks.decrementAndGet();
+        completionService.take().get();
+        if (listener != null) {
+          listener.checkInterrupt();
+        }
+      }
+    } catch (ExecutionException e) {

Review comment:
       find other exception to be caught




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

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


[GitHub] [hadoop] steveloughran commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
steveloughran commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r696704462



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -433,6 +435,31 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  /**
+   * Returns a ContentSummary instance containing the count of directories,
+   * files and total number of bytes under a given path
+   * @param path The given path
+   * @return ContentSummary
+   * @throws IOException if an error is encountered during listStatus calls
+   * or if there is any issue with the thread pool used while processing
+   */
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      TracingContext tracingContext = new TracingContext(clientCorrelationId,
+          fileSystemId, FSOperationType.GET_CONTENT_SUMMARY, true,
+          tracingHeaderFormat, listener);
+      return (new ContentSummaryProcessor(abfsStore)).getContentSummary(path,
+          tracingContext);
+    } catch (InterruptedException e) {
+      LOG.debug("Thread interrupted");
+      throw new IOException(e);

Review comment:
       InterruptedIOException

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public class ContentSummaryProcessor {

Review comment:
       nit: javadocs

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -433,6 +435,31 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  /**
+   * Returns a ContentSummary instance containing the count of directories,
+   * files and total number of bytes under a given path
+   * @param path The given path
+   * @return ContentSummary
+   * @throws IOException if an error is encountered during listStatus calls
+   * or if there is any issue with the thread pool used while processing
+   */
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      TracingContext tracingContext = new TracingContext(clientCorrelationId,
+          fileSystemId, FSOperationType.GET_CONTENT_SUMMARY, true,
+          tracingHeaderFormat, listener);
+      return (new ContentSummaryProcessor(abfsStore)).getContentSummary(path,
+          tracingContext);
+    } catch (InterruptedException e) {
+      LOG.debug("Thread interrupted");
+      throw new IOException(e);
+    } catch(ExecutionException ex) {
+      LOG.debug("GetContentSummary failed with error: {}", ex.getMessage());
+      throw new IOException(ex);

Review comment:
       prefer PathIOException with path included

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =

Review comment:
       Abfs Store to create the executor so that all ops which do async IO (this, block uploads, prefetch, openFile async...) can share a single pool

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =
+      new ExecutorCompletionService<>(executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+
+  /**
+   * Processes a given path for count of subdirectories, files and total number
+   * of bytes
+   * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
+   * listStatus calls to server
+   */
+  public ContentSummaryProcessor(ListingSupport abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path, TracingContext tracingContext)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path, tracingContext);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+          LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+              queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+      LOG.debug("Executor shutdown");
+    }
+    LOG.debug("Processed content summary of subtree under given path");
+    ContentSummary.Builder builder = new ContentSummary.Builder()
+        .directoryCount(directoryCount.get()).fileCount(fileCount.get())
+        .length(totalBytes.get()).spaceConsumed(totalBytes.get());
+    return builder.build();
+  }
+
+  /**
+   * Calls listStatus on given path and populated fileStatus queue with
+   * subdirectories. Is called by new tasks to process the complete subtree
+   * under a given path
+   * @param path: Path to a file or directory
+   * @throws IOException: listStatus error
+   * @throws InterruptedException: error while inserting into queue
+   */
+  private void processDirectoryTree(Path path, TracingContext tracingContext)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path, tracingContext);
+
+    for (FileStatus fileStatus : fileStatuses) {

Review comment:
       
   If you supported paged results, you could start queuing subdir work while still iterating through the list.
   
   In directories with many pages of results including directories, this could speed up processing as subdirectory scanning could start as soon as of the first page of results had been retrieved.
   

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,227 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_BUFFER_SIZE = 20;
+  private static final int FILES_PER_DIRECTORY = 2;
+  private static final int MAX_THREADS = 16;
+  private static final int NUM_FILES_FOR_LIST_MAX_TEST =
+      DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+private static final int NUM_CONCURRENT_CALLS = 8;
+
+  private final String[] directories = {"/testFolder",
+      "/testFolderII",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2",
+      "/testFolder/testFolder3",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7"};
+
+  private final byte[] b = new byte[TEST_BUFFER_SIZE];
+
+  public TestGetContentSummary() throws Exception {
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    createDirectoryStructure();
+    int fileCount = directories.length * FILES_PER_DIRECTORY;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    verifyContentSummary(contentSummary, directories.length, fileCount,
+        directories.length * TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testFileContentSummary() throws IOException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path filePath = new Path("/testFolder/testFile");
+    fs.create(filePath);
+    FSDataOutputStream out = fs.append(filePath);
+    out.write(b);
+    out.close();
+    ContentSummary contentSummary = fs.getContentSummary(filePath);
+    verifyContentSummary(contentSummary, 0, 1, TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testLeafDir() throws IOException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    fs.mkdirs(new Path("/testFolder/testFolder1"));
+    fs.mkdirs(new Path("/testFolder/testFolder2"));
+    Path leafDir = new Path("/testFolder/testFolder1/testFolder3");
+    fs.mkdirs(leafDir);
+    ContentSummary contentSummary = fs.getContentSummary(leafDir);
+    verifyContentSummary(contentSummary, 0, 0, 0);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesOnly()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path intermediateDir = new Path("/testFolder/testFolder1");
+    fs.mkdirs(intermediateDir);
+    populateDirWithFiles(intermediateDir, FILES_PER_DIRECTORY);
+    ContentSummary contentSummary =
+        fs.getContentSummary(intermediateDir);
+    verifyContentSummary(contentSummary, 0, FILES_PER_DIRECTORY,
+        TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesAndSubdirs()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path intermediateDir = new Path("/testFolder/testFolder1");
+    fs.mkdirs(intermediateDir);
+    populateDirWithFiles(intermediateDir, FILES_PER_DIRECTORY);
+    fs.mkdirs(new Path("/testFolder/testFolder1/testFolder3"));
+    fs.registerListener(
+        new TracingHeaderValidator(getConfiguration().getClientCorrelationId(),
+            fs.getFileSystemId(), FSOperationType.GET_CONTENT_SUMMARY, true,
+            0));
+    ContentSummary contentSummary =
+        fs.getContentSummary(intermediateDir);
+    verifyContentSummary(contentSummary, 1, FILES_PER_DIRECTORY,
+        TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testDirOverListMaxResultsItems()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path pathToListMaxDir = new Path("/testFolder/listMaxDir");
+    fs.mkdirs(pathToListMaxDir);
+    fs.mkdirs(new Path(pathToListMaxDir + "/testFolder2"));
+    populateDirWithFiles(pathToListMaxDir, NUM_FILES_FOR_LIST_MAX_TEST);
+    verifyContentSummary(
+        fs.getContentSummary(pathToListMaxDir), 1,
+        NUM_FILES_FOR_LIST_MAX_TEST, TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testInvalidPath() throws Exception {
+    AzureBlobFileSystem fs = getFileSystem();
+    intercept(IOException.class, () -> fs.getContentSummary(new Path(
+        "/nonExistentPath")));
+  }
+
+  @Test
+  public void testConcurrentGetContentSummaryCalls()
+      throws InterruptedException, ExecutionException, IOException {
+    AzureBlobFileSystem fs = getFileSystem();
+    ExecutorService executorService = new ThreadPoolExecutor(1, MAX_THREADS, 5,
+        TimeUnit.SECONDS, new SynchronousQueue<>());
+    CompletionService<ContentSummary> completionService =
+        new ExecutorCompletionService<>(executorService);
+    createDirectoryStructure();
+    for (int i = 0; i < NUM_CONCURRENT_CALLS; i++) {
+      completionService.submit(() -> fs.getContentSummary(new Path(
+          "/testFolder")));
+    }
+    for (int i = 0; i < NUM_CONCURRENT_CALLS; i++) {
+      ContentSummary contentSummary = completionService.take().get();
+      verifyContentSummary(contentSummary, 7, 8 * FILES_PER_DIRECTORY,
+          8 * TEST_BUFFER_SIZE);
+    }
+    executorService.shutdown();
+  }
+
+  private void verifyContentSummary(ContentSummary contentSummary,
+      long expectedDirectoryCount, long expectedFileCount, long expectedByteCount) {
+    Assertions.assertThat(contentSummary.getDirectoryCount())
+        .describedAs("Incorrect directory count").isEqualTo(expectedDirectoryCount);
+    Assertions.assertThat(contentSummary.getFileCount())
+        .describedAs("Incorrect file count").isEqualTo(expectedFileCount);
+    Assertions.assertThat(contentSummary.getLength())
+        .describedAs("Incorrect length").isEqualTo(expectedByteCount);
+    Assertions.assertThat(contentSummary.getSpaceConsumed())
+        .describedAs("Incorrect value of space consumed").isEqualTo(expectedByteCount);
+  }
+
+  private void createDirectoryStructure()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    for (String directory : directories) {
+      Path dirPath = new Path(directory);
+      fs.mkdirs(dirPath);
+      populateDirWithFiles(dirPath, FILES_PER_DIRECTORY);
+    }
+  }
+
+  private void populateDirWithFiles(Path directory, int numFiles)
+      throws ExecutionException, InterruptedException, IOException {
+    final List<Future<Void>> tasks = new ArrayList<>();
+    ExecutorService es = Executors.newFixedThreadPool(10);
+    for (int i = 0; i < numFiles; i++) {
+      final Path fileName = new Path(directory + "/test" + i);

Review comment:
       change to
   ```
   new Path(directory, String.format("test-%04d", i));.
   ```
   
   ensures ordering of paths.
   
   

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,227 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {

Review comment:
       if this runs against a real store, name needs to begin as ITest
    
   to make this a unit test, look at how ITestAbfsListStatusRemoteIterator returns a mock listing API.
   
   you could simply generate a deep/wide tree of results for listings to find, either by creating a tree of entries to simulate a directory tree; or programmatically creating them (e.g declaring that each dir has 10 subdirs and 100 files & so returning them)
   

##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,227 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_BUFFER_SIZE = 20;
+  private static final int FILES_PER_DIRECTORY = 2;
+  private static final int MAX_THREADS = 16;
+  private static final int NUM_FILES_FOR_LIST_MAX_TEST =
+      DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+private static final int NUM_CONCURRENT_CALLS = 8;
+
+  private final String[] directories = {"/testFolder",
+      "/testFolderII",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2",
+      "/testFolder/testFolder3",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7"};
+
+  private final byte[] b = new byte[TEST_BUFFER_SIZE];
+
+  public TestGetContentSummary() throws Exception {
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    createDirectoryStructure();
+    int fileCount = directories.length * FILES_PER_DIRECTORY;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    verifyContentSummary(contentSummary, directories.length, fileCount,
+        directories.length * TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testFileContentSummary() throws IOException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path filePath = new Path("/testFolder/testFile");
+    fs.create(filePath);

Review comment:
       1. ContentTestUtils has helper methods here
   2. `fs.create()` returns a stream you can write to




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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-750869008


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 37s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 43s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 22s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  2s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | -1 :x: |  javac  |   0m 29s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/8/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 1 new + 14 unchanged - 1 fixed = 15 total (was 15)  |
   | +1 :green_heart: |  compile  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  javac  |   0m 24s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/8/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 1 new + 8 unchanged - 1 fixed = 9 total (was 9)  |
   | +1 :green_heart: |  checkstyle  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 48s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  findbugs  |   1m  1s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/8/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 20s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  77m 55s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Exceptional return value of java.util.concurrent.ExecutorService.submit(Callable) ignored in org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor.processDirectoryTree(Path)  At ContentSummaryProcessor.java:ignored in org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor.processDirectoryTree(Path)  At ContentSummaryProcessor.java:[line 79] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/8/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 51b884af8bd0 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 16a20503cac |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/8/testReport/ |
   | Max. process+thread count | 720 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/8/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-758447893


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 35s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 11s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 29s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  0s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 19s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  0s |  |  patch has no errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 26s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/17/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 1 new + 17 unchanged - 0 fixed = 18 total (was 17)  |
   | -1 :x: |  javadoc  |   0m 25s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/17/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 1 new + 17 unchanged - 0 fixed = 18 total (was 17)  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 23s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  77m 46s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/17/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux c1a09b879fa9 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / b0eec090977 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/17/testReport/ |
   | Max. process+thread count | 689 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/17/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554585923



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,107 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)

Review comment:
       added log




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r552431610



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +367,21 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      ABFSContentSummary contentSummary =
+          (new ContentSummaryProcessor(abfsStore)).getContentSummary(path);
+      return new Builder().length(contentSummary.getLength())
+          .directoryCount(contentSummary.getDirectoryCount())
+          .fileCount(contentSummary.getFileCount())
+          .spaceConsumed(contentSummary.getSpaceConsumed()).build();
+    } catch (InterruptedException e) {
+      LOG.debug(e.toString());

Review comment:
       log interrupted




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-774697894


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 37s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  0s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 23s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 35s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 15s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  2s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/21/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  12m 50s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 59s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 51s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 29s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/21/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 16ad99234700 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 19ae0faacc5 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~20.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/21/testReport/ |
   | Max. process+thread count | 671 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/21/console |
   | versions | git=2.25.1 maven=3.6.3 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554927643



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+                queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+    }
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();
+      } else {
+        processFile(fileStatus);
+      }
+    }
+  }
+
+  private void processDirectory() {
+    directoryCount.incrementAndGet();
+  }
+
+  private void processFile(FileStatus fileStatus) {
+    fileCount.incrementAndGet();
+    totalBytes.addAndGet(fileStatus.getLen());
+  }
+
+  private synchronized void conditionalSubmitTaskToExecutor() {
+    if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) {
+      numTasks.incrementAndGet();

Review comment:
       See if there is a method for simple get, since you are not using the return value




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-758740529


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 33s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | -1 :x: |  mvninstall  |   6m 11s | [/branch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/artifact/out/branch-mvninstall-root.txt) |  root in trunk failed.  |
   | -1 :x: |  compile  |   1m 35s | [/branch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/artifact/out/branch-compile-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | -1 :x: |  compile  |   0m 16s | [/branch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/artifact/out/branch-compile-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-azure in trunk failed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.  |
   | +1 :green_heart: |  checkstyle  |   0m 20s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 36s |  |  trunk passed  |
   | -1 :x: |  shadedclient  |   2m  5s |  |  branch has errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 23s | [/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/artifact/out/branch-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-azure in trunk failed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  4s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  2s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 20s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | -1 :x: |  javac  |   0m 30s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 15 new + 0 unchanged - 0 fixed = 15 total (was 0)  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  javac  |   0m 27s | [/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/artifact/out/diff-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 generated 9 new + 0 unchanged - 0 fixed = 9 total (was 0)  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | -1 :x: |  shadedclient  |   0m 46s |  |  patch has errors when building and testing our client artifacts.  |
   | -1 :x: |  javadoc  |   0m 23s | [/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/artifact/out/diff-javadoc-javadoc-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 generated 17 new + 0 unchanged - 0 fixed = 17 total (was 0)  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 54s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 22s |  |  hadoop-azure in the patch passed.  |
   | +0 :ok: |  asflicense  |   0m 21s |  |  ASF License check generated no output?  |
   |  |   |  21m 55s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 20dbef9e7fe0 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 85b1c017eed |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/testReport/ |
   | Max. process+thread count | 259 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/19/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] sumangala-patki commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-745776042


   TEST RESULTS
   
   HNS Account Location: East US 2
   NonHNS Account Location: East US 2, Central US
   
   ```
   HNS OAuth
   
   [INFO] Tests run: 97, Failures: 0, Errors: 0, Skipped: 0
   [WARNING] Tests run: 462, Failures: 0, Errors: 0, Skipped: 68
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 24
   
   HNS SharedKey
   
   [INFO] Tests run: 97, Failures: 0, Errors: 0, Skipped: 0
   [WARNING] Tests run: 462, Failures: 0, Errors: 0, Skipped: 24
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 16
   
   Non-HNS SharedKey
   
   [INFO] Tests run: 97, Failures: 0, Errors: 0, Skipped: 0
   [WARNING] Tests run: 462, Failures: 0, Errors: 0, Skipped: 249
   [WARNING] Tests run: 208, Failures: 0, Errors: 0, Skipped: 16
   ```


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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-917874060


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 59s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 2 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 36s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 37s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   0m 59s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 34s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  16m 52s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | -1 :x: |  javac  |   0m 31s | [/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/37/artifact/out/results-compile-javac-hadoop-tools_hadoop-azure-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04.txt) |  hadoop-tools_hadoop-azure-jdkUbuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 generated 1 new + 25 unchanged - 1 fixed = 26 total (was 26)  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | -1 :x: |  javac  |   0m 26s | [/results-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/37/artifact/out/results-compile-javac-hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10.txt) |  hadoop-tools_hadoop-azure-jdkPrivateBuild-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 generated 1 new + 19 unchanged - 1 fixed = 20 total (was 20)  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 16s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 19s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  2s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  16m 39s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  5s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 30s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  79m 23s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/37/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 002837fb1840 4.15.0-143-generic #147-Ubuntu SMP Wed Apr 14 16:10:11 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 16d9436062f0da1fe4d074a011c2947697d72240 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/37/testReport/ |
   | Max. process+thread count | 593 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/37/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554934952



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,204 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private final String[] directories = {"/testFolder",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2", "/testFolder/testFolder3", "/testFolderII",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7",
+      "/testFolder/testFolder3/testFolder6/leafDir",
+      "/testFolderII/listMaxDir",
+      "/testFolderII/listMaxDir/" + DEFAULT_AZURE_LIST_MAX_RESULTS/2 + "_mid_folder"};
+
+  private final Path pathToFile = new Path("/testFolder/test1");
+  private final Path pathToListMaxDir = new Path("/testFolderII/listMaxDir");
+  private final Path pathToLeafDir =
+      new Path("/testFolder/testFolder3/testFolder6/leafDir");
+  private final Path pathToIntermediateDirWithFilesOnly = new Path(
+        "/testFolder/testFolder2/testFolder5");
+  private final Path pathToIntermediateDirWithFilesAndSubdirs = new Path(
+        "/testFolder/testFolder3");
+  private final String[] dirsWithNonEmptyFiles = {"/testFolder", "/testFolder/testFolder1",
+      "/testFolder/testFolder2/testFolder5", "/testFolder/testFolder3"};
+
+  private final AzureBlobFileSystem fs = createFileSystem();
+  private final int testBufferSize = 20;
+  private final int filesPerDirectory = 2;
+  private final int numFilesForListMaxTest = DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+  private final byte[] b = new byte[testBufferSize];
+  private final int maxThreads = 16;
+
+  public TestGetContentSummary() throws Exception {
+    createDirectoryStructure();
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException {
+    int fileCount =
+        (directories.length - 2) * filesPerDirectory + numFilesForListMaxTest;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    checkContentSummary(contentSummary, directories.length, fileCount,
+        dirsWithNonEmptyFiles.length * filesPerDirectory * testBufferSize);
+  }
+
+  @Test
+  public void testFileContentSummary() throws IOException {
+    ContentSummary contentSummary = fs.getContentSummary(pathToFile);
+    checkContentSummary(contentSummary, 0, 1, testBufferSize);
+  }
+
+  @Test
+  public void testLeafDir() throws IOException {
+    ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir);
+    checkContentSummary(contentSummary, 0, 0, 0);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesOnly() throws IOException {
+    ContentSummary contentSummary =
+        fs.getContentSummary(pathToIntermediateDirWithFilesOnly);
+    checkContentSummary(contentSummary, 0, filesPerDirectory,
+        testBufferSize * filesPerDirectory);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesAndSubdirs() throws IOException {
+    ContentSummary contentSummary =
+        fs.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs);
+    checkContentSummary(contentSummary, 3, 3 * filesPerDirectory,
+        testBufferSize * filesPerDirectory);
+  }
+
+  @Test
+  public void testDirOverListMaxResultsItems()
+      throws IOException {
+    checkContentSummary(
+        fs.getContentSummary(pathToListMaxDir), 1,
+        numFilesForListMaxTest + filesPerDirectory, 0);
+  }
+
+  @Test
+  public void testInvalidPath() throws Exception {
+    intercept(IOException.class, () -> fs.getContentSummary(new Path(
+        "/nonExistentPath")));
+  }
+
+  @Test
+  public void testConcurrentGetContentSummaryCalls()
+          throws InterruptedException, ExecutionException {
+    ExecutorService executorService = new ThreadPoolExecutor(1,
+            maxThreads, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+    ArrayList<Future<ContentSummary>> futures = new ArrayList<>();
+    for (String directory : directories) {
+      Future<ContentSummary> future = executorService.submit(
+              () -> fs.getContentSummary(new Path(directory)));
+      futures.add(future);
+    }
+    int[][] dirCS = {{8, 8 * filesPerDirectory, 8 * testBufferSize}, {0, filesPerDirectory, 2 * testBufferSize},

Review comment:
       move each array of elements into separate lines for better readability




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

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


[GitHub] [hadoop] vinaysbadami commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
vinaysbadami commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r553775292



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,107 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        completionService.take().get();
+        numTasks.decrementAndGet();
+      }
+    } finally {
+      numTasks.decrementAndGet();
+      executorService.shutdown();
+    }
+    executorService.awaitTermination(1, TimeUnit.SECONDS);

Review comment:
       if there was an exception, line 63 will not execute.

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,107 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        completionService.take().get();
+        numTasks.decrementAndGet();

Review comment:
       r we decrementing twice on the last task - once here and once in finally?




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r552435806



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.Listener;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor implements AutoCloseable {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Void>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+  private Listener listener = null;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0
+          || ((ThreadPoolExecutor) executorService).getActiveCount() > 0) {

Review comment:
       remove 3rd condition




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-757509492


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 34s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 38s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 36s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 30s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  5s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  4s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 25s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  2s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 21s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  78m 12s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/16/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 8ba355fb6378 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / ec22850dbef |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/16/testReport/ |
   | Max. process+thread count | 537 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/16/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554935350



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,204 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private final String[] directories = {"/testFolder",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2", "/testFolder/testFolder3", "/testFolderII",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7",
+      "/testFolder/testFolder3/testFolder6/leafDir",
+      "/testFolderII/listMaxDir",
+      "/testFolderII/listMaxDir/" + DEFAULT_AZURE_LIST_MAX_RESULTS/2 + "_mid_folder"};
+
+  private final Path pathToFile = new Path("/testFolder/test1");
+  private final Path pathToListMaxDir = new Path("/testFolderII/listMaxDir");
+  private final Path pathToLeafDir =
+      new Path("/testFolder/testFolder3/testFolder6/leafDir");
+  private final Path pathToIntermediateDirWithFilesOnly = new Path(
+        "/testFolder/testFolder2/testFolder5");
+  private final Path pathToIntermediateDirWithFilesAndSubdirs = new Path(
+        "/testFolder/testFolder3");
+  private final String[] dirsWithNonEmptyFiles = {"/testFolder", "/testFolder/testFolder1",
+      "/testFolder/testFolder2/testFolder5", "/testFolder/testFolder3"};
+
+  private final AzureBlobFileSystem fs = createFileSystem();
+  private final int testBufferSize = 20;
+  private final int filesPerDirectory = 2;
+  private final int numFilesForListMaxTest = DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+  private final byte[] b = new byte[testBufferSize];
+  private final int maxThreads = 16;
+
+  public TestGetContentSummary() throws Exception {
+    createDirectoryStructure();
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException {
+    int fileCount =
+        (directories.length - 2) * filesPerDirectory + numFilesForListMaxTest;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    checkContentSummary(contentSummary, directories.length, fileCount,
+        dirsWithNonEmptyFiles.length * filesPerDirectory * testBufferSize);
+  }
+
+  @Test
+  public void testFileContentSummary() throws IOException {
+    ContentSummary contentSummary = fs.getContentSummary(pathToFile);
+    checkContentSummary(contentSummary, 0, 1, testBufferSize);
+  }
+
+  @Test
+  public void testLeafDir() throws IOException {
+    ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir);
+    checkContentSummary(contentSummary, 0, 0, 0);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesOnly() throws IOException {
+    ContentSummary contentSummary =
+        fs.getContentSummary(pathToIntermediateDirWithFilesOnly);
+    checkContentSummary(contentSummary, 0, filesPerDirectory,
+        testBufferSize * filesPerDirectory);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesAndSubdirs() throws IOException {
+    ContentSummary contentSummary =
+        fs.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs);
+    checkContentSummary(contentSummary, 3, 3 * filesPerDirectory,
+        testBufferSize * filesPerDirectory);
+  }
+
+  @Test
+  public void testDirOverListMaxResultsItems()
+      throws IOException {
+    checkContentSummary(
+        fs.getContentSummary(pathToListMaxDir), 1,
+        numFilesForListMaxTest + filesPerDirectory, 0);
+  }
+
+  @Test
+  public void testInvalidPath() throws Exception {
+    intercept(IOException.class, () -> fs.getContentSummary(new Path(
+        "/nonExistentPath")));
+  }
+
+  @Test
+  public void testConcurrentGetContentSummaryCalls()
+          throws InterruptedException, ExecutionException {
+    ExecutorService executorService = new ThreadPoolExecutor(1,
+            maxThreads, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+    ArrayList<Future<ContentSummary>> futures = new ArrayList<>();
+    for (String directory : directories) {
+      Future<ContentSummary> future = executorService.submit(
+              () -> fs.getContentSummary(new Path(directory)));
+      futures.add(future);
+    }
+    int[][] dirCS = {{8, 8 * filesPerDirectory, 8 * testBufferSize}, {0, filesPerDirectory, 2 * testBufferSize},
+            {2, 3 * filesPerDirectory, 2 * testBufferSize}, {3, 3 * filesPerDirectory, 2 * testBufferSize},
+            {2, numFilesForListMaxTest + 2 * filesPerDirectory, 0}, {0, filesPerDirectory, 0},
+            {0, filesPerDirectory, filesPerDirectory * testBufferSize}, {1, filesPerDirectory, 0},
+            {0, filesPerDirectory, 0}, {0, 0, 0}, {1, numFilesForListMaxTest + 2, 0}, {0, filesPerDirectory, 0}};
+    executorService.shutdown();
+    for (int i=0; i<directories.length; i++) {
+      ContentSummary contentSummary = futures.get(i).get();
+      checkContentSummary(contentSummary, dirCS[i][0], dirCS[i][1], dirCS[i][2]);
+    }
+  }
+
+  private void checkContentSummary(ContentSummary contentSummary,

Review comment:
       long directoryCount, long fileCount, long byteCount
   rename to
   long expectedDirectoryCount, long expectedFileCount, long expectedByteCount




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r555527399



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,

Review comment:
       done




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-750941852


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 31s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 33s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 23s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 50s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 21s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  76m 45s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/9/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux a23ee008f0ae 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 005b854f6ba |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/9/testReport/ |
   | Max. process+thread count | 537 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/9/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] vinaysbadami commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
vinaysbadami commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548353156



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ContentSummary;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  //cached thread pool with custom max threads to avoid overloading
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (((ThreadPoolExecutor) executorService).getActiveCount() > 0) {
+      Thread.sleep(100);
+    }
+    executorService.shutdown();
+    return new ContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty()) {
+            executorService.submit(() -> {
+              try {
+                FileStatus fileStatus1;
+                while ((fileStatus1 = queue.poll(100, TimeUnit.MILLISECONDS)) != null) {

Review comment:
       I think we can still end up with < max threads in the following case:
   there is only 1 thread doing work, rest are waiting on queue.poll
   the thread is going thru files or the list responses are slow. it is yet to queue up any  subdir.
   The rest of the threads timeout and fall of their thread main
   

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ContentSummary;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  //cached thread pool with custom max threads to avoid overloading
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (((ThreadPoolExecutor) executorService).getActiveCount() > 0) {

Review comment:
       we should find a better way for this. the documentaion says it returns the approx number of active threads. For us the termination condiction is queue empty and no active threads. We should try to model that.




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548438235



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;

Review comment:
       done




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r696894388



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,227 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.fs.azurebfs.constants.FSOperationType;
+import org.apache.hadoop.fs.azurebfs.utils.TracingHeaderValidator;
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private static final int TEST_BUFFER_SIZE = 20;
+  private static final int FILES_PER_DIRECTORY = 2;
+  private static final int MAX_THREADS = 16;
+  private static final int NUM_FILES_FOR_LIST_MAX_TEST =
+      DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+private static final int NUM_CONCURRENT_CALLS = 8;
+
+  private final String[] directories = {"/testFolder",
+      "/testFolderII",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2",
+      "/testFolder/testFolder3",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7"};
+
+  private final byte[] b = new byte[TEST_BUFFER_SIZE];
+
+  public TestGetContentSummary() throws Exception {
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException, ExecutionException, InterruptedException {
+    AzureBlobFileSystem fs = getFileSystem();
+    createDirectoryStructure();
+    int fileCount = directories.length * FILES_PER_DIRECTORY;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    verifyContentSummary(contentSummary, directories.length, fileCount,
+        directories.length * TEST_BUFFER_SIZE);
+  }
+
+  @Test
+  public void testFileContentSummary() throws IOException {
+    AzureBlobFileSystem fs = getFileSystem();
+    fs.mkdirs(new Path("/testFolder"));
+    Path filePath = new Path("/testFolder/testFile");
+    fs.create(filePath);

Review comment:
       modified




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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r615684609



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -393,6 +397,32 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  /**
+   * Returns a ContentSummary instance containing the count of directories,
+   * files and total number of bytes under a given path
+   * @param path The given path
+   * @return ContentSummary
+   * @throws IOException if an error is encountered during listStatus calls
+   * or if there is any issue with the thread pool used while processing
+   */
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      ABFSContentSummary contentSummary =
+          (new ContentSummaryProcessor(abfsStore)).getContentSummary(path);

Review comment:
       ContentSummary class has extra members for storing and calculations related to snapshots of the directory structure, which are not needed here. Hence created a new class for ABFS




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-917867841


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 47s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 2 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  31m 55s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 44s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  4s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  6s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 25s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  5s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m  3s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 58s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  73m 21s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/36/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 21313d327f75 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / fa72f2db3bafabae5614556745aa68f11b7cafcc |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/36/testReport/ |
   | Max. process+thread count | 600 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/36/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554586230



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,107 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        completionService.take().get();
+        numTasks.decrementAndGet();
+      }
+    } finally {
+      numTasks.decrementAndGet();
+      executorService.shutdown();
+    }
+    executorService.awaitTermination(1, TimeUnit.SECONDS);
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);

Review comment:
       caught in the azureblobfilesystem getContentSummary method




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-916863901


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 53s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 2 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 34s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 45s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 40s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 31s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 45s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  9s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 51s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  15m 12s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 21s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/33/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 4 new + 3 unchanged - 1 fixed = 7 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m  8s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 46s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 10s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 36s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  76m 27s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/33/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux b4db7f4cd5bb 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 3039f7fd7ec82599ab5cdfaaad5cf74ce71bd700 |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/33/testReport/ |
   | Max. process+thread count | 684 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/33/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r696890740



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public class ContentSummaryProcessor {

Review comment:
       added description




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


[GitHub] [hadoop] surendralilhore commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
surendralilhore commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r614609260



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -393,6 +397,32 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  /**
+   * Returns a ContentSummary instance containing the count of directories,
+   * files and total number of bytes under a given path
+   * @param path The given path
+   * @return ContentSummary
+   * @throws IOException if an error is encountered during listStatus calls
+   * or if there is any issue with the thread pool used while processing
+   */
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      ABFSContentSummary contentSummary =
+          (new ContentSummaryProcessor(abfsStore)).getContentSummary(path);
+      return new Builder().length(contentSummary.getLength())
+          .directoryCount(contentSummary.getDirectoryCount())
+          .fileCount(contentSummary.getFileCount())
+          .spaceConsumed(contentSummary.getSpaceConsumed()).build();
+    } catch (InterruptedException e) {
+      LOG.debug("Thread interrupted");
+      throw new IOException(e);
+    } catch(ExecutionException ex) {
+      LOG.debug(ex.getCause().getMessage());

Review comment:
       Add some detail like "GetContentSummary failed with error"




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r696904612



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =
+      new ExecutorCompletionService<>(executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+
+  /**
+   * Processes a given path for count of subdirectories, files and total number
+   * of bytes
+   * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
+   * listStatus calls to server
+   */
+  public ContentSummaryProcessor(ListingSupport abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path, TracingContext tracingContext)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path, tracingContext);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+          LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+              queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+      LOG.debug("Executor shutdown");
+    }
+    LOG.debug("Processed content summary of subtree under given path");
+    ContentSummary.Builder builder = new ContentSummary.Builder()
+        .directoryCount(directoryCount.get()).fileCount(fileCount.get())
+        .length(totalBytes.get()).spaceConsumed(totalBytes.get());
+    return builder.build();
+  }
+
+  /**
+   * Calls listStatus on given path and populated fileStatus queue with
+   * subdirectories. Is called by new tasks to process the complete subtree
+   * under a given path
+   * @param path: Path to a file or directory
+   * @throws IOException: listStatus error
+   * @throws InterruptedException: error while inserting into queue
+   */
+  private void processDirectoryTree(Path path, TracingContext tracingContext)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path, tracingContext);
+
+    for (FileStatus fileStatus : fileStatuses) {

Review comment:
       yes, we should be able to incorporate that using the listiterator. Thanks, will make the change




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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548438691



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java
##########
@@ -0,0 +1,168 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs;
+
+import org.apache.hadoop.fs.ContentSummary;

Review comment:
       fixed




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

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


[GitHub] [hadoop] vinaysbadami commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
vinaysbadami commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r553852099



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,107 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        completionService.take().get();
+        numTasks.decrementAndGet();
+      }
+    } finally {
+      numTasks.decrementAndGet();
+      executorService.shutdown();
+    }
+    executorService.awaitTermination(1, TimeUnit.SECONDS);
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);

Review comment:
       what if this throws?




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-745807832


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |  26m 32s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m 22s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 35s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 12s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/3/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 48s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  1s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 23s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   | 103m 28s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/3/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 264536585afe 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / df35c7f5199 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/3/testReport/ |
   | Max. process+thread count | 535 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/3/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] hadoop-yetus removed a comment on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus removed a comment on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-811247660






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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548438503



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ContentSummary;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  //cached thread pool with custom max threads to avoid overloading
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (((ThreadPoolExecutor) executorService).getActiveCount() > 0) {
+      Thread.sleep(100);
+    }
+    executorService.shutdown();
+    return new ContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty()) {
+            executorService.submit(() -> {
+              try {
+                FileStatus fileStatus1;
+                while ((fileStatus1 = queue.poll(100, TimeUnit.MILLISECONDS)) != null) {
+                  processDirectoryTree(fileStatus1.getPath());
+                }
+              } catch (InterruptedException | IOException e) {
+                e.printStackTrace();

Review comment:
       done




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r555528100



##########
File path: hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java
##########
@@ -0,0 +1,204 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.assertj.core.api.Assertions;
+import org.junit.Test;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem;
+
+import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS;
+import static org.apache.hadoop.test.LambdaTestUtils.intercept;
+
+public class TestGetContentSummary extends AbstractAbfsIntegrationTest {
+
+  private final String[] directories = {"/testFolder",
+      "/testFolder/testFolder1",
+      "/testFolder/testFolder2", "/testFolder/testFolder3", "/testFolderII",
+      "/testFolder/testFolder2/testFolder4",
+      "/testFolder/testFolder2/testFolder5",
+      "/testFolder/testFolder3/testFolder6",
+      "/testFolder/testFolder3/testFolder7",
+      "/testFolder/testFolder3/testFolder6/leafDir",
+      "/testFolderII/listMaxDir",
+      "/testFolderII/listMaxDir/" + DEFAULT_AZURE_LIST_MAX_RESULTS/2 + "_mid_folder"};
+
+  private final Path pathToFile = new Path("/testFolder/test1");
+  private final Path pathToListMaxDir = new Path("/testFolderII/listMaxDir");
+  private final Path pathToLeafDir =
+      new Path("/testFolder/testFolder3/testFolder6/leafDir");
+  private final Path pathToIntermediateDirWithFilesOnly = new Path(
+        "/testFolder/testFolder2/testFolder5");
+  private final Path pathToIntermediateDirWithFilesAndSubdirs = new Path(
+        "/testFolder/testFolder3");
+  private final String[] dirsWithNonEmptyFiles = {"/testFolder", "/testFolder/testFolder1",
+      "/testFolder/testFolder2/testFolder5", "/testFolder/testFolder3"};
+
+  private final AzureBlobFileSystem fs = createFileSystem();
+  private final int testBufferSize = 20;
+  private final int filesPerDirectory = 2;
+  private final int numFilesForListMaxTest = DEFAULT_AZURE_LIST_MAX_RESULTS + 10;
+  private final byte[] b = new byte[testBufferSize];
+  private final int maxThreads = 16;
+
+  public TestGetContentSummary() throws Exception {
+    createDirectoryStructure();
+    new Random().nextBytes(b);
+  }
+
+  @Test
+  public void testFilesystemRoot()
+      throws IOException {
+    int fileCount =
+        (directories.length - 2) * filesPerDirectory + numFilesForListMaxTest;
+    ContentSummary contentSummary = fs.getContentSummary(new Path("/"));
+    checkContentSummary(contentSummary, directories.length, fileCount,
+        dirsWithNonEmptyFiles.length * filesPerDirectory * testBufferSize);
+  }
+
+  @Test
+  public void testFileContentSummary() throws IOException {
+    ContentSummary contentSummary = fs.getContentSummary(pathToFile);
+    checkContentSummary(contentSummary, 0, 1, testBufferSize);
+  }
+
+  @Test
+  public void testLeafDir() throws IOException {
+    ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir);
+    checkContentSummary(contentSummary, 0, 0, 0);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesOnly() throws IOException {
+    ContentSummary contentSummary =
+        fs.getContentSummary(pathToIntermediateDirWithFilesOnly);
+    checkContentSummary(contentSummary, 0, filesPerDirectory,
+        testBufferSize * filesPerDirectory);
+  }
+
+  @Test
+  public void testIntermediateDirWithFilesAndSubdirs() throws IOException {
+    ContentSummary contentSummary =
+        fs.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs);
+    checkContentSummary(contentSummary, 3, 3 * filesPerDirectory,
+        testBufferSize * filesPerDirectory);
+  }
+
+  @Test
+  public void testDirOverListMaxResultsItems()
+      throws IOException {
+    checkContentSummary(
+        fs.getContentSummary(pathToListMaxDir), 1,
+        numFilesForListMaxTest + filesPerDirectory, 0);
+  }
+
+  @Test
+  public void testInvalidPath() throws Exception {
+    intercept(IOException.class, () -> fs.getContentSummary(new Path(
+        "/nonExistentPath")));
+  }
+
+  @Test
+  public void testConcurrentGetContentSummaryCalls()
+          throws InterruptedException, ExecutionException {
+    ExecutorService executorService = new ThreadPoolExecutor(1,
+            maxThreads, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+    ArrayList<Future<ContentSummary>> futures = new ArrayList<>();
+    for (String directory : directories) {
+      Future<ContentSummary> future = executorService.submit(
+              () -> fs.getContentSummary(new Path(directory)));
+      futures.add(future);
+    }
+    int[][] dirCS = {{8, 8 * filesPerDirectory, 8 * testBufferSize}, {0, filesPerDirectory, 2 * testBufferSize},

Review comment:
       done




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-745138292


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 33s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 38s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 40s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 16s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 56s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 18s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/1/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 16 new + 2 unchanged - 0 fixed = 18 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m 54s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  2s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 21s |  |  hadoop-azure in the patch passed.  |
   | -1 :x: |  asflicense  |   0m 33s | [/patch-asflicense-problems.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/1/artifact/out/patch-asflicense-problems.txt) |  The patch generated 4 ASF License warnings.  |
   |  |   |  77m 17s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/1/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux a3e72be523de 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 5bf977e6b16 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/1/testReport/ |
   | Max. process+thread count | 666 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/1/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-750817856


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 31s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 58s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  15m 58s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 30s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 19s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 32s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/6/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 5 new + 2 unchanged - 0 fixed = 7 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 42s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | -1 :x: |  findbugs  |   1m  1s | [/new-findbugs-hadoop-tools_hadoop-azure.html](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/6/artifact/out/new-findbugs-hadoop-tools_hadoop-azure.html) |  hadoop-tools/hadoop-azure generated 1 new + 0 unchanged - 0 fixed = 1 total (was 0)  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 30s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  76m 38s |  |  |
   
   
   | Reason | Tests |
   |-------:|:------|
   | FindBugs | module:hadoop-tools/hadoop-azure |
   |  |  Exceptional return value of java.util.concurrent.ExecutorService.submit(Callable) ignored in org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor.processDirectoryTree(Path)  At ContentSummaryProcessor.java:ignored in org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor.processDirectoryTree(Path)  At ContentSummaryProcessor.java:[line 78] |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/6/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 4313da151cc1 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 16a20503cac |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/6/testReport/ |
   | Max. process+thread count | 735 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/6/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] vinaysbadami commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
vinaysbadami commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554757241



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",

Review comment:
       should we add the log at lines 70, 74 and 76




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548952240



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,123 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Object>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int NUM_THREADS = 16;
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (!queue.isEmpty() || numTasks.get() > 0) {
+      for (Future<Object> future : futures) {
+        try {
+          future.get(10, TimeUnit.MILLISECONDS);
+          futures.remove(future);
+        } catch (TimeoutException ignored) {
+        } catch (ExecutionException e) {
+          LOG.debug(e.toString());
+          throw new IOException(e);
+        }
+      }
+    }
+
+    executorService.shutdown();
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) {
+            numTasks.incrementAndGet();
+            Future<Object> future = executorService.submit(() -> {
+              FileStatus fileStatus1;
+              while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS))
+                  != null) {

Review comment:
       It is thrown when we call get() on that particular future returned; i.e., thrown by the main process while it is waiting for threads to complete tasks




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-839703482


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 38s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 14s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 39s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 58s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m  7s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 26s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 27s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/results-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/29/artifact/out/results-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 5 new + 2 unchanged - 0 fixed = 7 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 21s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  1s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m  7s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  4s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  76m 10s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/29/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 39718d2cb922 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / aa48086d7d7abd4fef4a037f684fe99cb68797fc |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/29/testReport/ |
   | Max. process+thread count | 614 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/29/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548360780



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ContentSummary;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  //cached thread pool with custom max threads to avoid overloading
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (((ThreadPoolExecutor) executorService).getActiveCount() > 0) {
+      Thread.sleep(100);
+    }
+    executorService.shutdown();
+    return new ContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty()) {
+            executorService.submit(() -> {
+              try {
+                FileStatus fileStatus1;
+                while ((fileStatus1 = queue.poll(100, TimeUnit.MILLISECONDS)) != null) {
+                  processDirectoryTree(fileStatus1.getPath());
+                }
+              } catch (InterruptedException | IOException e) {
+                e.printStackTrace();

Review comment:
       Same as abve comment, remove printstacktrace. rethrow as IOException.




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

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


[GitHub] [hadoop] vinaysbadami commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
vinaysbadami commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548448441



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final AtomicInteger NUM_TASKS = new AtomicInteger(0);
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while(!queue.isEmpty() || NUM_TASKS.get() > 0) {
+      Thread.sleep(10);
+    }
+    executorService.shutdown();
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);

Review comment:
       we should move the try catch across the whole method since listing can also throw

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final AtomicInteger NUM_TASKS = new AtomicInteger(0);

Review comment:
       follow naming conventions for variables

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final AtomicInteger NUM_TASKS = new AtomicInteger(0);
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while(!queue.isEmpty() || NUM_TASKS.get() > 0) {
+      Thread.sleep(10);
+    }
+    executorService.shutdown();
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty() && NUM_TASKS.get() < NUM_THREADS) {
+            NUM_TASKS.incrementAndGet();
+            executorService.submit(() -> {
+              try {
+                FileStatus fileStatus1;
+                while ((fileStatus1 = queue.poll(100, TimeUnit.MILLISECONDS))
+                    != null) {
+                  processDirectoryTree(fileStatus1.getPath());
+                }
+                NUM_TASKS.decrementAndGet();

Review comment:
       is a better place for this at the end of this method? say line 98




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r552455938



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.Listener;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor implements AutoCloseable {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Void>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+  private Listener listener = null;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0
+          || ((ThreadPoolExecutor) executorService).getActiveCount() > 0) {
+        numTasks.decrementAndGet();
+        completionService.take().get();
+        if (listener != null) {
+          listener.checkInterrupt();
+        }
+      }
+    } catch (ExecutionException e) {
+      LOG.debug(e.getMessage());
+      throw new IOException(e);
+    } finally {
+      executorService.shutdown();
+      if (listener != null) {
+        listener.checkShutdown(((ThreadPoolExecutor)executorService).getActiveCount());
+      }
+    }
+
+//    close();
+    executorService.shutdownNow();
+    if (listener != null) {
+      // statement reachable only when no exceptions thrown by threads
+      listener.checkAllTasksComplete(numTasks,
+          ((ThreadPoolExecutor)executorService).getActiveCount());
+    }
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+    if (listener != null) {
+      synchronized (this) {
+        listener.verifyThreadCount(numTasks,
+            (ThreadPoolExecutor) executorService, NUM_THREADS);
+      }
+    }
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) {
+            numTasks.incrementAndGet();
+            Future<Void> future = completionService.submit(() -> {
+              FileStatus fileStatus1;
+              while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS))
+                  != null) {
+                processDirectoryTree(fileStatus1.getPath());
+              }
+              if (listener != null) {
+                synchronized (this) {
+                  if (!listener.isInterrupted() && listener.shouldInterrupt()) {
+                    listener.setInterrupted();
+                    throw new InterruptedException();
+                  }
+                }
+              }
+              return null;
+            });
+            futures.add(future);
+          }
+        }
+      } else {
+        processFile(fileStatus);
+      }
+    }
+  }
+
+  private void processDirectory() {
+    directoryCount.incrementAndGet();
+  }
+
+  private void processFile(FileStatus fileStatus) {
+    fileCount.incrementAndGet();
+    totalBytes.addAndGet(fileStatus.getLen());
+  }
+
+  @Override
+  public void close() {

Review comment:
       find out if needed
   do shutdown once




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548437756



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ContentSummary;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  //cached thread pool with custom max threads to avoid overloading
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (((ThreadPoolExecutor) executorService).getActiveCount() > 0) {

Review comment:
       replaced termination condition with queue empty and atomic int to track number of tasks




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r552436232



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,169 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.Listener;
+import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor implements AutoCloseable {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Void>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+  private Listener listener = null;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    try {
+      while (!queue.isEmpty() || numTasks.get() > 0
+          || ((ThreadPoolExecutor) executorService).getActiveCount() > 0) {
+        numTasks.decrementAndGet();

Review comment:
       decrement in finally




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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554925918



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+                queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+    }
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();
+      } else {
+        processFile(fileStatus);
+      }
+    }
+  }
+
+  private void processDirectory() {
+    directoryCount.incrementAndGet();
+  }
+
+  private void processFile(FileStatus fileStatus) {
+    fileCount.incrementAndGet();
+    totalBytes.addAndGet(fileStatus.getLen());
+  }
+
+  private synchronized void conditionalSubmitTaskToExecutor() {
+    if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) {

Review comment:
       rename to MAX_THREAD_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.

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-757104733


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 33s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  7s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 28s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 25s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 33s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 58s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/14/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 2 new + 2 unchanged - 0 fixed = 4 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  1s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   0m 59s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 22s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 34s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  77m 46s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/14/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 07224f808847 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 890f2da6244 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/14/testReport/ |
   | Max. process+thread count | 548 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/14/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r552431187



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +367,21 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      ABFSContentSummary contentSummary =
+          (new ContentSummaryProcessor(abfsStore)).getContentSummary(path);
+      return new Builder().length(contentSummary.getLength())
+          .directoryCount(contentSummary.getDirectoryCount())
+          .fileCount(contentSummary.getFileCount())
+          .spaceConsumed(contentSummary.getSpaceConsumed()).build();
+    } catch (InterruptedException e) {
+      LOG.debug(e.toString());
+      throw new IOException(e.getMessage());

Review comment:
       pass e as inner Exception




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-758598253


   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 29s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | -1 :x: |  mvninstall  |  27m 42s | [/branch-mvninstall-root.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/18/artifact/out/branch-mvninstall-root.txt) |  root in trunk failed.  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 32s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 24s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 58s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 27s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   1m  2s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  trunk passed  |
   | -0 :warning: |  patch  |   1m 18s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 37s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 27s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 28s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 22s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  15m  1s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 24s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  0s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 31s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  72m 42s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/18/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux f09c96db14e7 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / ca7dd5fad33 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/18/testReport/ |
   | Max. process+thread count | 536 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/18/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-839710241


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 37s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  35m 16s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 37s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 25s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 38s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 28s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  0s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 41s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  14m 59s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 31s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 31s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 27s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 27s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  1s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  3s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 15s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  4s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 32s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  76m 45s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/30/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux fb458e0feff1 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / f320785b00896eb8a59bc4ce764f09788a23b307 |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/30/testReport/ |
   | Max. process+thread count | 549 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/30/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r696889981



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -433,6 +435,31 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  /**
+   * Returns a ContentSummary instance containing the count of directories,
+   * files and total number of bytes under a given path
+   * @param path The given path
+   * @return ContentSummary
+   * @throws IOException if an error is encountered during listStatus calls
+   * or if there is any issue with the thread pool used while processing
+   */
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      TracingContext tracingContext = new TracingContext(clientCorrelationId,
+          fileSystemId, FSOperationType.GET_CONTENT_SUMMARY, true,
+          tracingHeaderFormat, listener);
+      return (new ContentSummaryProcessor(abfsStore)).getContentSummary(path,
+          tracingContext);
+    } catch (InterruptedException e) {
+      LOG.debug("Thread interrupted");
+      throw new IOException(e);

Review comment:
       changed




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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r696892660



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,147 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.azurebfs.utils.TracingContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =

Review comment:
       done
   Changes: abfsStore creates thread pool executor; each individual getContentSummary call creates a separate completionService instance (which uses the common thread pool)




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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-747284265


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 30s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |   |   0m  0s | [test4tests](test4tests) |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  32m 46s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 38s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  checkstyle  |   0m 27s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 39s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m  0s |  |  branch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 32s |  |  trunk passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +0 :ok: |  spotbugs  |   0m 59s |  |  Used deprecated FindBugs config; considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   0m 57s |  |  trunk passed  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 30s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javac  |   0m 30s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 26s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  javac  |   0m 26s |  |  the patch passed  |
   | -0 :warning: |  checkstyle  |   0m 17s | [/diff-checkstyle-hadoop-tools_hadoop-azure.txt](https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/4/artifact/out/diff-checkstyle-hadoop-tools_hadoop-azure.txt) |  hadoop-tools/hadoop-azure: The patch generated 1 new + 2 unchanged - 0 fixed = 3 total (was 2)  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  |  The patch has no whitespace issues.  |
   | +1 :green_heart: |  shadedclient  |  14m 39s |  |  patch has no errors when building and testing our client artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  the patch passed with JDK Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01  |
   | +1 :green_heart: |  findbugs  |   1m  1s |  |  the patch passed  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   1m 31s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 33s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  76m 29s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/4/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 251867f51016 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 4c033bafa02 |
   | Default Java | Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.9.1+1-Ubuntu-0ubuntu1.18.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_275-8u275-b01-0ubuntu1~18.04-b01 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/4/testReport/ |
   | Max. process+thread count | 537 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/4/console |
   | versions | git=2.17.1 maven=3.6.0 findbugs=4.0.6 |
   | Powered by | Apache Yetus 0.13.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548494909



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,109 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final AtomicInteger NUM_TASKS = new AtomicInteger(0);
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while(!queue.isEmpty() || NUM_TASKS.get() > 0) {
+      Thread.sleep(10);
+    }
+    executorService.shutdown();
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty() && NUM_TASKS.get() < NUM_THREADS) {
+            NUM_TASKS.incrementAndGet();
+            executorService.submit(() -> {
+              try {
+                FileStatus fileStatus1;
+                while ((fileStatus1 = queue.poll(100, TimeUnit.MILLISECONDS))
+                    != null) {
+                  processDirectoryTree(fileStatus1.getPath());
+                }
+                NUM_TASKS.decrementAndGet();

Review comment:
       yes, moved to finally




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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548360232



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;

Review comment:
       import order 
   java
   nonjava
   apache




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

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


[GitHub] [hadoop] vinaysbadami commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
vinaysbadami commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r553781281



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,107 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)

Review comment:
       see if u need to log for debugging purposes - numtasks, length of queue etc etc




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: HADOOP-17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-917863650


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 43s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  0s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 2 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  33m  9s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 44s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  compile  |   0m 41s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  checkstyle  |   0m 31s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 42s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 31s |  |  trunk passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 10s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  14m 50s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  15m 12s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 34s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javac  |   0m 34s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 21s |  |  hadoop-tools/hadoop-azure: The patch generated 0 new + 3 unchanged - 1 fixed = 3 total (was 4)  |
   | +1 :green_heart: |  mvnsite  |   0m 33s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 25s |  |  the patch passed with JDK Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 23s |  |  the patch passed with JDK Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10  |
   | +1 :green_heart: |  spotbugs  |   1m 11s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  14m 39s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m 10s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 37s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  76m 42s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/34/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux d8fc1a88eb29 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / 8259a2e9ec9b4423d221a5c683d4e72ae9fb18aa |
   | Default Java | Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.11+9-Ubuntu-0ubuntu2.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_292-8u292-b10-0ubuntu1~20.04-b10 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/34/testReport/ |
   | Max. process+thread count | 549 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/34/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548437586



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ContentSummary;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS,
+      5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  //cached thread pool with custom max threads to avoid overloading
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (((ThreadPoolExecutor) executorService).getActiveCount() > 0) {
+      Thread.sleep(100);
+    }
+    executorService.shutdown();
+    return new ContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty()) {
+            executorService.submit(() -> {
+              try {
+                FileStatus fileStatus1;
+                while ((fileStatus1 = queue.poll(100, TimeUnit.MILLISECONDS)) != null) {

Review comment:
       will work when using executor service




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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554927643



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+                queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+    }
+
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();
+      } else {
+        processFile(fileStatus);
+      }
+    }
+  }
+
+  private void processDirectory() {
+    directoryCount.incrementAndGet();
+  }
+
+  private void processFile(FileStatus fileStatus) {
+    fileCount.incrementAndGet();
+    totalBytes.addAndGet(fileStatus.getLen());
+  }
+
+  private synchronized void conditionalSubmitTaskToExecutor() {
+    if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) {
+      numTasks.incrementAndGet();

Review comment:
       See if there is a method for simple increment, since you are not using the return value




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-838295912


   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime |  Logfile | Comment |
   |:----:|----------:|--------:|:--------:|:-------:|
   | +0 :ok: |  reexec  |   0m 58s |  |  Docker mode activated.  |
   |||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  |  No case conflicting files found.  |
   | +0 :ok: |  codespell  |   0m  1s |  |  codespell was not available.  |
   | +1 :green_heart: |  @author  |   0m  0s |  |  The patch does not contain any @author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  |  The patch appears to include 1 new or modified test files.  |
   |||| _ trunk Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |  36m 37s |  |  trunk passed  |
   | +1 :green_heart: |  compile  |   0m 36s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  checkstyle  |   0m 22s |  |  trunk passed  |
   | +1 :green_heart: |  mvnsite  |   0m 36s |  |  trunk passed  |
   | +1 :green_heart: |  javadoc  |   0m 29s |  |  trunk passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 26s |  |  trunk passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   0m 59s |  |  trunk passed  |
   | +1 :green_heart: |  shadedclient  |  16m 30s |  |  branch has no errors when building and testing our client artifacts.  |
   | -0 :warning: |  patch  |  16m 47s |  |  Used diff version of patch file. Binary files and potentially other changes not applied. Please rebase and squash commits if necessary.  |
   |||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 29s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javac  |   0m 29s |  |  the patch passed  |
   | +1 :green_heart: |  compile  |   0m 25s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  javac  |   0m 25s |  |  the patch passed  |
   | +1 :green_heart: |  blanks  |   0m  0s |  |  The patch has no blanks issues.  |
   | +1 :green_heart: |  checkstyle  |   0m 17s |  |  the patch passed  |
   | +1 :green_heart: |  mvnsite  |   0m 28s |  |  the patch passed  |
   | +1 :green_heart: |  javadoc  |   0m 22s |  |  the patch passed with JDK Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04  |
   | +1 :green_heart: |  javadoc  |   0m 20s |  |  the patch passed with JDK Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08  |
   | +1 :green_heart: |  spotbugs  |   1m  2s |  |  the patch passed  |
   | +1 :green_heart: |  shadedclient  |  16m 50s |  |  patch has no errors when building and testing our client artifacts.  |
   |||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   2m  4s |  |  hadoop-azure in the patch passed.  |
   | +1 :green_heart: |  asflicense  |   0m 30s |  |  The patch does not generate ASF License warnings.  |
   |  |   |  82m 18s |  |  |
   
   
   | Subsystem | Report/Notes |
   |----------:|:-------------|
   | Docker | ClientAPI=1.41 ServerAPI=1.41 base: https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/28/artifact/out/Dockerfile |
   | GITHUB PR | https://github.com/apache/hadoop/pull/2549 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall mvnsite unit shadedclient spotbugs checkstyle codespell |
   | uname | Linux 3f0f70abbc0f 4.15.0-101-generic #102-Ubuntu SMP Mon May 11 10:07:26 UTC 2020 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | dev-support/bin/hadoop.sh |
   | git revision | trunk / fa34b5788479095b23df3a7f9a34f6592c45f6ed |
   | Default Java | Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   | Multi-JDK versions | /usr/lib/jvm/java-11-openjdk-amd64:Ubuntu-11.0.10+9-Ubuntu-0ubuntu1.20.04 /usr/lib/jvm/java-8-openjdk-amd64:Private Build-1.8.0_282-8u282-b08-0ubuntu1~20.04-b08 |
   |  Test Results | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/28/testReport/ |
   | Max. process+thread count | 519 (vs. ulimit of 5500) |
   | modules | C: hadoop-tools/hadoop-azure U: hadoop-tools/hadoop-azure |
   | Console output | https://ci-hadoop.apache.org/job/hadoop-multibranch/job/PR-2549/28/console |
   | versions | git=2.25.1 maven=3.6.3 spotbugs=4.2.2 |
   | Powered by | Apache Yetus 0.14.0-SNAPSHOT https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


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

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


[GitHub] [hadoop] bilaharith commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
bilaharith commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r554922036



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +367,22 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path path) throws IOException {
+    try {
+      ABFSContentSummary contentSummary =
+          (new ContentSummaryProcessor(abfsStore)).getContentSummary(path);
+      return new Builder().length(contentSummary.getLength())
+          .directoryCount(contentSummary.getDirectoryCount())
+          .fileCount(contentSummary.getFileCount())
+          .spaceConsumed(contentSummary.getSpaceConsumed()).build();
+    } catch (InterruptedException | ExecutionException e) {

Review comment:
       split into 2 catch blocks




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

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


[GitHub] [hadoop] surendralilhore commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
surendralilhore commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r614671618



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,144 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =
+      new ExecutorCompletionService<>(executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+
+  /**
+   * Processes a given path for count of subdirectories, files and total number
+   * of bytes
+   * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
+   * listStatus calls to server
+   */
+  public ContentSummaryProcessor(ListingSupport abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+          LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+              queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+      LOG.debug("Executor shutdown");
+    }
+    LOG.debug("Processed content summary of subtree under given path");
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  /**
+   * Calls listStatus on given path and populated fileStatus queue with
+   * subdirectories. Is called by new tasks to process the complete subtree
+   * under a given path
+   * @param path: Path to a file or directory
+   * @throws IOException: listStatus error
+   * @throws InterruptedException: error while inserting into queue
+   */
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();

Review comment:
       Why don't you utilize ForkJoinPool like [HDFS-13768](https://github.com/apache/hadoop/blob/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java#L240), so you no need to worry about submitting task after traverse.




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

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


[GitHub] [hadoop] hadoop-yetus commented on pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
hadoop-yetus commented on pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#issuecomment-756614368






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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548438017



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +366,20 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {

Review comment:
       done

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java
##########
@@ -363,6 +366,20 @@ public boolean delete(final Path f, final boolean recursive) throws IOException
 
   }
 
+  @Override
+  public ContentSummary getContentSummary(Path f) throws IOException {
+    org.apache.hadoop.fs.azurebfs.utils.ContentSummary contentSummary = null;
+    try {
+      contentSummary = (new ContentSummaryProcessor(abfsStore)).getContentSummary(f);
+    } catch (InterruptedException e) {
+      e.printStackTrace();

Review comment:
       done




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

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


[GitHub] [hadoop] sumangala-patki commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
sumangala-patki commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r555527632



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,119 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private static final int NUM_THREADS = 16;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final CompletionService<Void> completionService = new ExecutorCompletionService<>(
+      executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",

Review comment:
       added




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

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


[GitHub] [hadoop] surendralilhore commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
surendralilhore commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r614661413



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,144 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private static final int CORE_POOL_SIZE = 1;
+  private static final int MAX_THREAD_COUNT = 16;
+  private static final int KEEP_ALIVE_TIME = 5;
+  private static final int POLL_TIMEOUT = 100;
+  private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final ListingSupport abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(
+      CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS,
+      new SynchronousQueue<>());
+  private final CompletionService<Void> completionService =
+      new ExecutorCompletionService<>(executorService);
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+
+  /**
+   * Processes a given path for count of subdirectories, files and total number
+   * of bytes
+   * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
+   * listStatus calls to server
+   */
+  public ContentSummaryProcessor(ListingSupport abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+          throws IOException, ExecutionException, InterruptedException {
+    try {
+      processDirectoryTree(path);
+      while (!queue.isEmpty() || numTasks.get() > 0) {
+        try {
+          completionService.take().get();
+        } finally {
+          numTasks.decrementAndGet();
+          LOG.debug("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}",
+              queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount());
+        }
+      }
+    } finally {
+      executorService.shutdownNow();
+      LOG.debug("Executor shutdown");
+    }
+    LOG.debug("Processed content summary of subtree under given path");
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  /**
+   * Calls listStatus on given path and populated fileStatus queue with
+   * subdirectories. Is called by new tasks to process the complete subtree
+   * under a given path
+   * @param path: Path to a file or directory
+   * @throws IOException: listStatus error
+   * @throws InterruptedException: error while inserting into queue
+   */
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        conditionalSubmitTaskToExecutor();

Review comment:
       What if still some item queued and there is not directory to traverse, who will process remaining queued item ?




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

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


[GitHub] [hadoop] vinaysbadami commented on a change in pull request #2549: Hadoop 17428. ABFS: Implementation for getContentSummary

Posted by GitBox <gi...@apache.org>.
vinaysbadami commented on a change in pull request #2549:
URL: https://github.com/apache/hadoop/pull/2549#discussion_r548867756



##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,123 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Object>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int NUM_THREADS = 16;
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (!queue.isEmpty() || numTasks.get() > 0) {
+      for (Future<Object> future : futures) {
+        try {
+          future.get(10, TimeUnit.MILLISECONDS);
+          futures.remove(future);
+        } catch (TimeoutException ignored) {
+        } catch (ExecutionException e) {
+          LOG.debug(e.toString());
+          throw new IOException(e);
+        }
+      }
+    }
+
+    executorService.shutdown();
+    return new ABFSContentSummary(totalBytes.get(), directoryCount.get(),
+        fileCount.get(), totalBytes.get());
+  }
+
+  private void processDirectoryTree(Path path)
+      throws IOException, InterruptedException {
+    FileStatus[] fileStatuses = abfsStore.listStatus(path);
+
+    for (FileStatus fileStatus : fileStatuses) {
+      if (fileStatus.isDirectory()) {
+        queue.put(fileStatus);
+        processDirectory();
+        synchronized (this) {
+          if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) {
+            numTasks.incrementAndGet();
+            Future<Object> future = executorService.submit(() -> {
+              FileStatus fileStatus1;
+              while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS))
+                  != null) {

Review comment:
       what happens to an exception generated here?

##########
File path: hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java
##########
@@ -0,0 +1,123 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.azurebfs.services;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore;
+import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary;
+
+public class ContentSummaryProcessor {
+  private final AtomicLong fileCount = new AtomicLong(0L);
+  private final AtomicLong directoryCount = new AtomicLong(0L);
+  private final AtomicLong totalBytes = new AtomicLong(0L);
+  private final AtomicInteger numTasks = new AtomicInteger(0);
+  private final AzureBlobFileSystemStore abfsStore;
+  private final ExecutorService executorService = new ThreadPoolExecutor(1,
+      NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>());
+  private final LinkedBlockingQueue<FileStatus> queue = new LinkedBlockingQueue<>();
+  private final Set<Future<Object>> futures =
+      Collections.newSetFromMap(new ConcurrentHashMap<>());
+  private static final Logger LOG =
+      LoggerFactory.getLogger(ContentSummaryProcessor.class);
+  private static final int NUM_THREADS = 16;
+  private static final int POLL_TIMEOUT = 100;
+
+  public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) {
+    this.abfsStore = abfsStore;
+  }
+
+  public ABFSContentSummary getContentSummary(Path path)
+      throws IOException, InterruptedException {
+    processDirectoryTree(path);
+
+    while (!queue.isEmpty() || numTasks.get() > 0) {
+      for (Future<Object> future : futures) {
+        try {
+          future.get(10, TimeUnit.MILLISECONDS);

Review comment:
       why needed given we r counting numTasks




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

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