2929import java .util .concurrent .atomic .AtomicInteger ;
3030import java .util .concurrent .atomic .AtomicLong ;
3131
32- import org .apache .hadoop .fs .azurebfs .utils .TracingContext ;
3332import org .slf4j .Logger ;
3433import org .slf4j .LoggerFactory ;
3534
3635import org .apache .hadoop .fs .ContentSummary ;
3736import org .apache .hadoop .fs .FileStatus ;
3837import org .apache .hadoop .fs .Path ;
38+ import org .apache .hadoop .fs .azurebfs .AzureBlobFileSystemStore ;
39+ import org .apache .hadoop .fs .azurebfs .utils .TracingContext ;
3940
41+ /**
42+ * Class to carry out parallelized recursive listing on a given path to
43+ * collect directory and file count/size information, as part of the
44+ * implementation for the Filesystem method getContentSummary
45+ */
4046public class ContentSummaryProcessor {
4147 private static final int MAX_THREAD_COUNT = 16 ;
4248 private static final int POLL_TIMEOUT = 100 ;
@@ -56,33 +62,34 @@ public class ContentSummaryProcessor {
5662 * @param abfsStore Instance of AzureBlobFileSystemStore, used to make
5763 * listStatus calls to server
5864 */
59- public ContentSummaryProcessor (ListingSupport abfsStore ,
60- ExecutorService executorService ) {
65+ public ContentSummaryProcessor (ListingSupport abfsStore ) {
6166 this .abfsStore = abfsStore ;
62- this .executorService = executorService ;
67+ this .executorService = (( AzureBlobFileSystemStore ) abfsStore ). getContentSummaryExecutorService () ;
6368 completionService = new ExecutorCompletionService <>(this .executorService );
6469 }
6570
66- public ContentSummary getContentSummary (Path path , TracingContext tracingContext )
67- throws IOException , ExecutionException , InterruptedException {
68- try {
69- processDirectoryTree (path , tracingContext );
70- while (!queue .isEmpty () || numTasks .get () > 0 ) {
71- try {
72- completionService .take ().get ();
73- } finally {
74- numTasks .decrementAndGet ();
75- LOG .debug ("FileStatus queue size = {}, number of submitted unfinished tasks = {}, active thread count = {}" ,
76- queue .size (), numTasks , ((ThreadPoolExecutor ) executorService ).getActiveCount ());
77- }
71+ public ContentSummary getContentSummary (Path path ,
72+ TracingContext tracingContext )
73+ throws IOException , ExecutionException , InterruptedException {
74+
75+ processDirectoryTree (path , tracingContext );
76+ while (!queue .isEmpty () || numTasks .get () > 0 ) {
77+ try {
78+ completionService .take ().get ();
79+ } finally {
80+ numTasks .decrementAndGet ();
81+ LOG .debug (
82+ "FileStatus queue size = {}, number of submitted unfinished tasks"
83+ + " = {}, active thread count = {}" ,
84+ queue .size (), numTasks ,
85+ ((ThreadPoolExecutor ) executorService ).getActiveCount ());
7886 }
79- } finally {
80- executorService .shutdownNow ();
81- LOG .debug ("Executor shutdown" );
8287 }
88+
8389 LOG .debug ("Processed content summary of subtree under given path" );
84- ContentSummary .Builder builder = new ContentSummary .Builder ()
85- .directoryCount (directoryCount .get ()).fileCount (fileCount .get ())
90+ ContentSummary .Builder builder =
91+ new ContentSummary .Builder ().directoryCount (
92+ directoryCount .get ()).fileCount (fileCount .get ())
8693 .length (totalBytes .get ()).spaceConsumed (totalBytes .get ());
8794 return builder .build ();
8895 }
0 commit comments