From 87088567b5b5efe31f30980d16042bbc00fae190 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Fri, 11 Dec 2020 16:52:14 +0530 Subject: [PATCH 01/33] add getContentSummary and prelim test --- .../fs/azurebfs/AzureBlobFileSystem.java | 12 +++ .../services/ContentSummaryProcessor.java | 99 +++++++++++++++++++ .../fs/azurebfs/services/ProcessingQueue.java | 66 +++++++++++++ .../fs/azurebfs/utils/ContentSummary.java | 15 +++ .../fs/azurebfs/TestGetContentSummary.java | 64 ++++++++++++ 5 files changed, 256 insertions(+) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 75522462bb0e7..b1b35af7c0907 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -37,6 +37,9 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.ContentSummary.Builder; +import org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -363,6 +366,15 @@ 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 = + (new ContentSummaryProcessor()).getContentSummary(abfsStore, f); + return new Builder().length(contentSummary.length) + .directoryCount(contentSummary.directoryCount).fileCount(contentSummary.fileCount) + .spaceConsumed(contentSummary.spaceConsumed).build(); + } + @Override public FileStatus[] listStatus(final Path f) throws IOException { LOG.debug( diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java new file mode 100644 index 0000000000000..bdf3844f79f65 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -0,0 +1,99 @@ +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.atomic.AtomicLong; + +public class ContentSummaryProcessor { + private AtomicLong fileCount = new AtomicLong(0L); + private AtomicLong directoryCount = new AtomicLong(0L); + private AtomicLong totalBytes = new AtomicLong(0L); + private ProcessingQueue queue = new ProcessingQueue(); + private AzureBlobFileSystemStore abfsStore; + private static final int NUM_THREADS = 16; + + public ContentSummaryProcessor() { + } + + public ContentSummary getContentSummary(AzureBlobFileSystemStore abfsStore, + Path path) throws IOException { + this.abfsStore = abfsStore; + FileStatus fileStatus = abfsStore.getFileStatus(path); + if (!fileStatus.isDirectory()) { + processFile(fileStatus); + } else { + this.queue.add(fileStatus); + Thread[] threads = new Thread[16]; + + for(int i = 0; i < NUM_THREADS; ++i) { + threads[i] = new Thread(new ContentSummaryProcessor.ThreadProcessor()); + threads[i].start(); + } + + for (Thread t : threads) { + try { + t.join(); + } catch (InterruptedException var10) { + Thread.currentThread().interrupt(); + } + } + } + + return new ContentSummary(totalBytes.get(), directoryCount.get(), fileCount.get(), totalBytes.get()); + } + + private void processDirectoryTree(Path path) throws IOException { + FileStatus[] fileStatuses = enumerateDirectoryInternal(path.toString()); + if (fileStatuses == null || fileStatuses.length == 0) { + return; + } + for (FileStatus fileStatus : fileStatuses) { + System.out.println(fileStatus.getPath().toString()); + if (fileStatus.isDirectory()) { + this.queue.add(fileStatus); + this.processDirectory(); + System.out.println("===== incrementing dir count!!!"); + } else { + this.processFile(fileStatus); + } + } + } + + private void processDirectory() { + this.directoryCount.incrementAndGet(); + System.out.println("----- dir count is " + directoryCount); + } + + private void processFile(FileStatus fileStatus) { + this.fileCount.incrementAndGet(); + this.totalBytes.addAndGet(fileStatus.getLen()); + } + + private FileStatus[] enumerateDirectoryInternal(String path) throws IOException { + return abfsStore.listStatus(new Path(path)); //try catch if needed + } + + private class ThreadProcessor implements Runnable { + private ThreadProcessor() { + } + + public void run() { + try { + FileStatus fileStatus; + while((fileStatus = ContentSummaryProcessor.this.queue.poll()) != null) { + if (fileStatus.isDirectory()) { + ContentSummaryProcessor.this.processDirectoryTree(fileStatus.getPath()); + } + ContentSummaryProcessor.this.queue.unregister(); + } + + } catch (IOException var9) { + throw new RuntimeException("IOException processing Directory tree", var9); + } + } + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java new file mode 100644 index 0000000000000..90dca3b71511f --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java @@ -0,0 +1,66 @@ +package org.apache.hadoop.fs.azurebfs.services; + +import java.util.LinkedList; +import java.util.Queue; + +public class ProcessingQueue { + + private final Queue internalQueue = new LinkedList(); + private int processorCount = 0; + + ProcessingQueue() { + } + + public synchronized void add(T item) { + if (item == null) { + throw new IllegalArgumentException("Cannot put null into queue"); + } else { + this.internalQueue.add(item); + this.notifyAll(); + } + } + + public synchronized T poll() { + while(true) { + try { + if (this.isQueueEmpty() && !this.done()) { + this.wait(); + continue; + } + + if (!this.isQueueEmpty()) { + ++this.processorCount; + return this.internalQueue.poll(); + } + + if (this.done()) { + return null; + } + } catch (InterruptedException var2) { + Thread.currentThread().interrupt(); + } + + return null; + } + } + + public synchronized void unregister() { + --this.processorCount; + if (this.processorCount < 0) { + throw new IllegalStateException("too many unregister()'s. processorCount is now " + this.processorCount); + } else { + if (this.done()) { + this.notifyAll(); + } + + } + } + + private boolean done() { + return this.processorCount == 0 && this.isQueueEmpty(); + } + + private boolean isQueueEmpty() { + return this.internalQueue.peek() == null; + } +} diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java new file mode 100644 index 0000000000000..8073147b142e5 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java @@ -0,0 +1,15 @@ +package org.apache.hadoop.fs.azurebfs.utils; + +public class ContentSummary{ + public final long length; + public final long directoryCount; + public final long fileCount; + public final long spaceConsumed; + + public ContentSummary(long length, long directoryCount, long fileCount, long spaceConsumed) { + this.length = length; + this.directoryCount = directoryCount; + this.fileCount = fileCount; + this.spaceConsumed = spaceConsumed; + } +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java new file mode 100644 index 0000000000000..3317ef1d35b3b --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java @@ -0,0 +1,64 @@ +package org.apache.hadoop.fs.azurebfs; + +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.Path; +import org.assertj.core.api.Assertions; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +public class TestGetContentSummary extends AbstractAbfsIntegrationTest { + private final String[] directories = {"testFolder", "testFolder/testFolder1", + "testFolder/testFolder2", "testFolder/testFolder3", "testFolder" + + "/testFolder2/testFolder4", "testFolder/testFolder2/testFolder5", + "testFolder/testFolder3/testFolder6", "testFolder/testFolder3/testFolder7"}; + private final int filesPerDirectory = 10; + + public TestGetContentSummary() throws Exception { + } + + @Test + public void testFileFolderCount() + throws IOException, ExecutionException, InterruptedException { + AzureBlobFileSystem fs = getFileSystem(); + createDirectoryStructure(); + ContentSummary contentSummary = fs.getContentSummary(new Path("/")); + System.out.println(contentSummary.toString()); + Assertions.assertThat(contentSummary.getDirectoryCount()) + .describedAs("Directory count does not match") + .isEqualTo(directories.length); + Assertions.assertThat(contentSummary.getFileCount()) + .describedAs("File count incorrect") + .isEqualTo(directories.length * filesPerDirectory); + } + + void createDirectoryStructure() + throws IOException, ExecutionException, InterruptedException { + AzureBlobFileSystem fs = getFileSystem(); + for (String directory : directories) { + fs.mkdirs(new Path("/" + directory)); + } + + for (String directory : directories) { + final List> tasks = new ArrayList<>(); + ExecutorService es = Executors.newFixedThreadPool(10); + for (int i = 0; i < filesPerDirectory; i++) { + final Path fileName = new Path("/" + directory + "/test" + i); + tasks.add(es.submit(() -> { + touch(fileName); + return null; + })); + } + for (Future task : tasks) { + task.get(); + } + es.shutdownNow(); + } + } +} From 01aba061116e845b463ff873e0d8fe9f3d95b6ea Mon Sep 17 00:00:00 2001 From: Sumangala Date: Mon, 14 Dec 2020 16:58:36 +0530 Subject: [PATCH 02/33] remove gfs call --- .../fs/azurebfs/AzureBlobFileSystem.java | 2 +- .../services/ContentSummaryProcessor.java | 64 +++++++------------ .../fs/azurebfs/TestGetContentSummary.java | 35 ++++++++-- 3 files changed, 52 insertions(+), 49 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index b1b35af7c0907..e09ee809ddddf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -369,7 +369,7 @@ 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 = - (new ContentSummaryProcessor()).getContentSummary(abfsStore, f); + (new ContentSummaryProcessor(abfsStore)).getContentSummary(f); return new Builder().length(contentSummary.length) .directoryCount(contentSummary.directoryCount).fileCount(contentSummary.fileCount) .spaceConsumed(contentSummary.spaceConsumed).build(); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index bdf3844f79f65..2e185331a1ec5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -9,54 +9,42 @@ import java.util.concurrent.atomic.AtomicLong; public class ContentSummaryProcessor { - private AtomicLong fileCount = new AtomicLong(0L); - private AtomicLong directoryCount = new AtomicLong(0L); - private AtomicLong totalBytes = new AtomicLong(0L); - private ProcessingQueue queue = new ProcessingQueue(); - private AzureBlobFileSystemStore abfsStore; + private final AtomicLong fileCount = new AtomicLong(0L); + private final AtomicLong directoryCount = new AtomicLong(0L); + private final AtomicLong totalBytes = new AtomicLong(0L); + private final ProcessingQueue queue = new ProcessingQueue<>(); + private final AzureBlobFileSystemStore abfsStore; private static final int NUM_THREADS = 16; - public ContentSummaryProcessor() { + public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { + this.abfsStore = abfsStore; } - public ContentSummary getContentSummary(AzureBlobFileSystemStore abfsStore, - Path path) throws IOException { - this.abfsStore = abfsStore; - FileStatus fileStatus = abfsStore.getFileStatus(path); - if (!fileStatus.isDirectory()) { - processFile(fileStatus); - } else { - this.queue.add(fileStatus); - Thread[] threads = new Thread[16]; + public ContentSummary getContentSummary(Path path) throws IOException { + processDirectoryTree(path); + Thread[] threads = new Thread[16]; - for(int i = 0; i < NUM_THREADS; ++i) { - threads[i] = new Thread(new ContentSummaryProcessor.ThreadProcessor()); - threads[i].start(); - } + for(int i = 0; i < NUM_THREADS; ++i) { + threads[i] = new Thread(new ContentSummaryProcessor.ThreadProcessor()); + threads[i].start(); + } - for (Thread t : threads) { - try { - t.join(); - } catch (InterruptedException var10) { - Thread.currentThread().interrupt(); - } + for (Thread t : threads) { + try { + t.join(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); } } - return new ContentSummary(totalBytes.get(), directoryCount.get(), fileCount.get(), totalBytes.get()); } private void processDirectoryTree(Path path) throws IOException { - FileStatus[] fileStatuses = enumerateDirectoryInternal(path.toString()); - if (fileStatuses == null || fileStatuses.length == 0) { - return; - } + FileStatus[] fileStatuses = abfsStore.listStatus(path); for (FileStatus fileStatus : fileStatuses) { - System.out.println(fileStatus.getPath().toString()); if (fileStatus.isDirectory()) { - this.queue.add(fileStatus); this.processDirectory(); - System.out.println("===== incrementing dir count!!!"); + this.queue.add(fileStatus); } else { this.processFile(fileStatus); } @@ -65,7 +53,6 @@ private void processDirectoryTree(Path path) throws IOException { private void processDirectory() { this.directoryCount.incrementAndGet(); - System.out.println("----- dir count is " + directoryCount); } private void processFile(FileStatus fileStatus) { @@ -73,10 +60,6 @@ private void processFile(FileStatus fileStatus) { this.totalBytes.addAndGet(fileStatus.getLen()); } - private FileStatus[] enumerateDirectoryInternal(String path) throws IOException { - return abfsStore.listStatus(new Path(path)); //try catch if needed - } - private class ThreadProcessor implements Runnable { private ThreadProcessor() { } @@ -90,9 +73,8 @@ public void run() { } ContentSummaryProcessor.this.queue.unregister(); } - - } catch (IOException var9) { - throw new RuntimeException("IOException processing Directory tree", var9); + } catch (IOException e) { + throw new RuntimeException("IOException processing Directory tree", e); } } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java index 3317ef1d35b3b..df3f3659e0e1f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java @@ -15,19 +15,28 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { private final String[] directories = {"testFolder", "testFolder/testFolder1", - "testFolder/testFolder2", "testFolder/testFolder3", "testFolder" - + "/testFolder2/testFolder4", "testFolder/testFolder2/testFolder5", + "testFolder/testFolder2", "testFolder/testFolder3", "testFolderII", + "testFolder/testFolder2/testFolder4", "testFolder/testFolder2/testFolder5", "testFolder/testFolder3/testFolder6", "testFolder/testFolder3/testFolder7"}; - private final int filesPerDirectory = 10; + private final int filesPerDirectory = 2; + private final AzureBlobFileSystem fs = createFileSystem(); public TestGetContentSummary() throws Exception { + createDirectoryStructure(); } + @Test + public void testFileContentSummary() throws IOException { + ContentSummary contentSummary = fs.getContentSummary( + new Path("/"+directories[2] + "/test0")); + Assertions.assertThat(contentSummary.getFileCount()) + .isEqualTo(1); + } + + @Test public void testFileFolderCount() - throws IOException, ExecutionException, InterruptedException { - AzureBlobFileSystem fs = getFileSystem(); - createDirectoryStructure(); + throws IOException { ContentSummary contentSummary = fs.getContentSummary(new Path("/")); System.out.println(contentSummary.toString()); Assertions.assertThat(contentSummary.getDirectoryCount()) @@ -38,9 +47,21 @@ public void testFileFolderCount() .isEqualTo(directories.length * filesPerDirectory); } + @Test + public void testEmptyDir() throws IOException { + Path pathToEmptyDir = new Path("/emptyDir"); + fs.mkdirs(pathToEmptyDir); + ContentSummary contentSummary = + fs.getContentSummary(pathToEmptyDir); + Assertions.assertThat(contentSummary.getFileCount()) + .isEqualTo(0); + Assertions.assertThat(contentSummary.getDirectoryCount()) + .isEqualTo(0); + } + void createDirectoryStructure() throws IOException, ExecutionException, InterruptedException { - AzureBlobFileSystem fs = getFileSystem(); +// AzureBlobFileSystem fs = getFileSystem(); for (String directory : directories) { fs.mkdirs(new Path("/" + directory)); } From 2876a8b535f603ef273c6154aee89fdfa9f5ff4c Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 15 Dec 2020 12:41:05 +0530 Subject: [PATCH 03/33] add tests --- .../fs/azurebfs/TestGetContentSummary.java | 99 +++++++++++++++---- 1 file changed, 78 insertions(+), 21 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java index df3f3659e0e1f..2c0fdfb791ed4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java @@ -1,13 +1,16 @@ package org.apache.hadoop.fs.azurebfs; import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.P; import org.assertj.core.api.Assertions; import org.junit.Test; import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.Random; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -25,47 +28,101 @@ public TestGetContentSummary() throws Exception { createDirectoryStructure(); } + @Test + public void testFilesystemRoot() + throws IOException { + ContentSummary contentSummary = fs.getContentSummary(new Path("/")); + System.out.println(contentSummary.toString()); + checkContentSummary(contentSummary, directories.length, + directories.length * filesPerDirectory, 0); + } + @Test public void testFileContentSummary() throws IOException { - ContentSummary contentSummary = fs.getContentSummary( - new Path("/"+directories[2] + "/test0")); - Assertions.assertThat(contentSummary.getFileCount()) - .isEqualTo(1); + Path filePath = new Path("/testFolderII/testFile"); + FSDataOutputStream out = fs.create(filePath); + byte[] b = new byte[20]; + new Random().nextBytes(b); + out.write(b); + out.hsync(); + ContentSummary contentSummary = fs.getContentSummary(filePath); + checkContentSummary(contentSummary, 0, 1, 20); } + @Test + public void testLeafDir() throws IOException { + Path pathToLeafDir = + new Path("/testFolder/testFolder2/testFolder4" + + "/leafDir"); + fs.mkdirs(pathToLeafDir); + ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir); + checkContentSummary(contentSummary, 0, 0, 0); + } @Test - public void testFileFolderCount() - throws IOException { - ContentSummary contentSummary = fs.getContentSummary(new Path("/")); - System.out.println(contentSummary.toString()); - Assertions.assertThat(contentSummary.getDirectoryCount()) - .describedAs("Directory count does not match") - .isEqualTo(directories.length); - Assertions.assertThat(contentSummary.getFileCount()) - .describedAs("File count incorrect") - .isEqualTo(directories.length * filesPerDirectory); + public void testIntermediateDirWithFilesOnly() throws IOException { + String dirPath = "/testFolder/testFolder3/testFolder6"; + byte[] b = new byte[20]; + new Random().nextBytes(b); + for (int i = 0; i < filesPerDirectory; i++) { + FSDataOutputStream out = fs.append(new Path(dirPath + "/test" + i)); + out.write(b); + out.close(); + } + ContentSummary contentSummary = fs.getContentSummary(new Path(dirPath)); + checkContentSummary(contentSummary, 0, filesPerDirectory, 20 * filesPerDirectory); + } + + @Test + public void testIntermediateDirWithFilesAndSubdirs() throws IOException { + Path dirPath = new Path("/testFolder/testFolder3"); + byte[] b = new byte[20]; + new Random().nextBytes(b); + for (int i = 0; i < filesPerDirectory; i++) { + FSDataOutputStream out = fs.append(new Path(dirPath + "/test" + i)); + out.write(b); + out.close(); + } + Path dir2Path = new Path("/testFolder/testFolder3/testFolder6"); + for (int i = 0; i < filesPerDirectory; i++) { + FSDataOutputStream out = fs.append(new Path(dir2Path + "/test" + i)); + out.write(b); + out.close(); + } + ContentSummary contentSummary = fs.getContentSummary(dirPath); + checkContentSummary(contentSummary, 2, 3 * filesPerDirectory, 20 * 2 * 2); } @Test public void testEmptyDir() throws IOException { - Path pathToEmptyDir = new Path("/emptyDir"); + Path pathToEmptyDir = new Path("/testFolder/emptyDir"); fs.mkdirs(pathToEmptyDir); ContentSummary contentSummary = fs.getContentSummary(pathToEmptyDir); - Assertions.assertThat(contentSummary.getFileCount()) - .isEqualTo(0); + checkContentSummary(contentSummary, 0, 0, 0); + } + + private void checkContentSummary(ContentSummary contentSummary, + long directoryCount, long fileCount, long byteCount) { Assertions.assertThat(contentSummary.getDirectoryCount()) - .isEqualTo(0); + .describedAs("Incorrect directory count") + .isEqualTo(directoryCount); + Assertions.assertThat(contentSummary.getFileCount()) + .describedAs("Incorrect file count") + .isEqualTo(fileCount); + Assertions.assertThat(contentSummary.getLength()) + .describedAs("Incorrect length") + .isEqualTo(byteCount); + Assertions.assertThat(contentSummary.getSpaceConsumed()) + .describedAs("Incorrect value of space consumed") + .isEqualTo(byteCount); } - void createDirectoryStructure() + private void createDirectoryStructure() throws IOException, ExecutionException, InterruptedException { -// AzureBlobFileSystem fs = getFileSystem(); for (String directory : directories) { fs.mkdirs(new Path("/" + directory)); } - for (String directory : directories) { final List> tasks = new ArrayList<>(); ExecutorService es = Executors.newFixedThreadPool(10); From a9960da2259d53171fbc51285815dc9e4c223439 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 15 Dec 2020 23:10:55 +0530 Subject: [PATCH 04/33] pr draft --- .../services/ContentSummaryProcessor.java | 29 ++++- .../fs/azurebfs/services/ProcessingQueue.java | 104 +++++++++------- .../fs/azurebfs/utils/ContentSummary.java | 23 +++- .../fs/azurebfs/TestGetContentSummary.java | 112 +++++++++++------- 4 files changed, 175 insertions(+), 93 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 2e185331a1ec5..59c2845ba90a1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.services; import org.apache.hadoop.fs.FileStatus; @@ -24,7 +42,7 @@ public ContentSummary getContentSummary(Path path) throws IOException { processDirectoryTree(path); Thread[] threads = new Thread[16]; - for(int i = 0; i < NUM_THREADS; ++i) { + for (int i = 0; i < NUM_THREADS; ++i) { threads[i] = new Thread(new ContentSummaryProcessor.ThreadProcessor()); threads[i].start(); } @@ -36,7 +54,8 @@ public ContentSummary getContentSummary(Path path) throws IOException { Thread.currentThread().interrupt(); } } - return new ContentSummary(totalBytes.get(), directoryCount.get(), fileCount.get(), totalBytes.get()); + return new ContentSummary(totalBytes.get(), directoryCount.get(), + fileCount.get(), totalBytes.get()); } private void processDirectoryTree(Path path) throws IOException { @@ -67,9 +86,11 @@ private ThreadProcessor() { public void run() { try { FileStatus fileStatus; - while((fileStatus = ContentSummaryProcessor.this.queue.poll()) != null) { + while ((fileStatus = ContentSummaryProcessor.this.queue.poll()) + != null) { if (fileStatus.isDirectory()) { - ContentSummaryProcessor.this.processDirectoryTree(fileStatus.getPath()); + ContentSummaryProcessor.this + .processDirectoryTree(fileStatus.getPath()); } ContentSummaryProcessor.this.queue.unregister(); } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java index 90dca3b71511f..3e7d5ae07a24d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java @@ -1,3 +1,21 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.services; import java.util.LinkedList; @@ -5,62 +23,58 @@ public class ProcessingQueue { - private final Queue internalQueue = new LinkedList(); - private int processorCount = 0; + private final Queue internalQueue = new LinkedList<>(); + private int processorCount = 0; - ProcessingQueue() { - } + ProcessingQueue() { + } - public synchronized void add(T item) { - if (item == null) { - throw new IllegalArgumentException("Cannot put null into queue"); - } else { - this.internalQueue.add(item); - this.notifyAll(); - } + public synchronized void add(T item) { + if (item == null) { + throw new IllegalArgumentException("Cannot put null into queue"); + } else { + this.internalQueue.add(item); + this.notifyAll(); } + } - public synchronized T poll() { - while(true) { - try { - if (this.isQueueEmpty() && !this.done()) { - this.wait(); - continue; - } - - if (!this.isQueueEmpty()) { - ++this.processorCount; - return this.internalQueue.poll(); - } - - if (this.done()) { - return null; - } - } catch (InterruptedException var2) { - Thread.currentThread().interrupt(); + public synchronized T poll() { + while (true) { + try { + if (this.isQueueEmpty() && !this.done()) { + this.wait(); + continue; + } + if (!this.isQueueEmpty()) { + ++this.processorCount; + return this.internalQueue.poll(); } - return null; + } catch (InterruptedException var2) { + Thread.currentThread().interrupt(); } + return null; } + } - public synchronized void unregister() { - --this.processorCount; - if (this.processorCount < 0) { - throw new IllegalStateException("too many unregister()'s. processorCount is now " + this.processorCount); - } else { - if (this.done()) { - this.notifyAll(); - } - + public synchronized void unregister() { + --this.processorCount; + if (this.processorCount < 0) { + throw new IllegalStateException( + "too many unregister()'s. processorCount is now " + + this.processorCount); + } else { + if (this.done()) { + this.notifyAll(); } } + } - private boolean done() { - return this.processorCount == 0 && this.isQueueEmpty(); - } + private boolean done() { + return this.processorCount == 0 && this.isQueueEmpty(); + } - private boolean isQueueEmpty() { - return this.internalQueue.peek() == null; - } + private boolean isQueueEmpty() { + return this.internalQueue.peek() == null; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java index 8073147b142e5..dd557614033cb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java @@ -1,12 +1,31 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs.utils; -public class ContentSummary{ +public class ContentSummary { public final long length; public final long directoryCount; public final long fileCount; public final long spaceConsumed; - public ContentSummary(long length, long directoryCount, long fileCount, long spaceConsumed) { + public ContentSummary(long length, long directoryCount, long fileCount, + long spaceConsumed) { this.length = length; this.directoryCount = directoryCount; this.fileCount = fileCount; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java index 2c0fdfb791ed4..82a50fd8a95dd 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java @@ -1,9 +1,26 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.fs.azurebfs; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.yarn.webapp.hamlet2.Hamlet.P; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -16,23 +33,27 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS; + 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/testFolder2/testFolder4", + "testFolder/testFolder2/testFolder5", + "testFolder/testFolder3/testFolder6", + "testFolder/testFolder3/testFolder7"}; private final int filesPerDirectory = 2; private final AzureBlobFileSystem fs = createFileSystem(); + private final byte[] b = new byte[20]; public TestGetContentSummary() throws Exception { createDirectoryStructure(); + new Random().nextBytes(b); } @Test - public void testFilesystemRoot() - throws IOException { + public void testFilesystemRoot() throws IOException { ContentSummary contentSummary = fs.getContentSummary(new Path("/")); - System.out.println(contentSummary.toString()); checkContentSummary(contentSummary, directories.length, directories.length * filesPerDirectory, 0); } @@ -41,19 +62,16 @@ public void testFilesystemRoot() public void testFileContentSummary() throws IOException { Path filePath = new Path("/testFolderII/testFile"); FSDataOutputStream out = fs.create(filePath); - byte[] b = new byte[20]; - new Random().nextBytes(b); out.write(b); - out.hsync(); + out.close(); ContentSummary contentSummary = fs.getContentSummary(filePath); checkContentSummary(contentSummary, 0, 1, 20); } @Test public void testLeafDir() throws IOException { - Path pathToLeafDir = - new Path("/testFolder/testFolder2/testFolder4" - + "/leafDir"); + Path pathToLeafDir = new Path( + "/testFolder/testFolder2/testFolder4" + "/leafDir"); fs.mkdirs(pathToLeafDir); ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir); checkContentSummary(contentSummary, 0, 0, 0); @@ -62,22 +80,19 @@ public void testLeafDir() throws IOException { @Test public void testIntermediateDirWithFilesOnly() throws IOException { String dirPath = "/testFolder/testFolder3/testFolder6"; - byte[] b = new byte[20]; - new Random().nextBytes(b); for (int i = 0; i < filesPerDirectory; i++) { FSDataOutputStream out = fs.append(new Path(dirPath + "/test" + i)); out.write(b); out.close(); } ContentSummary contentSummary = fs.getContentSummary(new Path(dirPath)); - checkContentSummary(contentSummary, 0, filesPerDirectory, 20 * filesPerDirectory); + checkContentSummary(contentSummary, 0, filesPerDirectory, + 20 * filesPerDirectory); } @Test public void testIntermediateDirWithFilesAndSubdirs() throws IOException { Path dirPath = new Path("/testFolder/testFolder3"); - byte[] b = new byte[20]; - new Random().nextBytes(b); for (int i = 0; i < filesPerDirectory; i++) { FSDataOutputStream out = fs.append(new Path(dirPath + "/test" + i)); out.write(b); @@ -97,46 +112,59 @@ public void testIntermediateDirWithFilesAndSubdirs() throws IOException { public void testEmptyDir() throws IOException { Path pathToEmptyDir = new Path("/testFolder/emptyDir"); fs.mkdirs(pathToEmptyDir); - ContentSummary contentSummary = - fs.getContentSummary(pathToEmptyDir); + ContentSummary contentSummary = fs.getContentSummary(pathToEmptyDir); checkContentSummary(contentSummary, 0, 0, 0); } + @Test + public void testDirOverListMaxResultsItems() + throws IOException, ExecutionException, InterruptedException { + Path pathToDir = new Path("/testFolder/testFolder2/maxListDir"); + fs.mkdirs(pathToDir); + populateDirWithFiles(pathToDir, DEFAULT_AZURE_LIST_MAX_RESULTS + 100); + FSDataOutputStream out = fs.append(new Path(pathToDir + "/test0")); + out.write(b); + out.close(); + checkContentSummary( + fs.getContentSummary(new Path("/testFolder" + "/testFolder2")), 3, + DEFAULT_AZURE_LIST_MAX_RESULTS + 100 + filesPerDirectory * 3, 20); + } + private void checkContentSummary(ContentSummary contentSummary, long directoryCount, long fileCount, long byteCount) { Assertions.assertThat(contentSummary.getDirectoryCount()) - .describedAs("Incorrect directory count") - .isEqualTo(directoryCount); + .describedAs("Incorrect directory count").isEqualTo(directoryCount); Assertions.assertThat(contentSummary.getFileCount()) - .describedAs("Incorrect file count") - .isEqualTo(fileCount); + .describedAs("Incorrect file count").isEqualTo(fileCount); Assertions.assertThat(contentSummary.getLength()) - .describedAs("Incorrect length") - .isEqualTo(byteCount); + .describedAs("Incorrect length").isEqualTo(byteCount); Assertions.assertThat(contentSummary.getSpaceConsumed()) - .describedAs("Incorrect value of space consumed") - .isEqualTo(byteCount); + .describedAs("Incorrect value of space consumed").isEqualTo(byteCount); } private void createDirectoryStructure() throws IOException, ExecutionException, InterruptedException { for (String directory : directories) { - fs.mkdirs(new Path("/" + directory)); + Path dirPath = new Path("/" + directory); + fs.mkdirs(dirPath); + populateDirWithFiles(dirPath, filesPerDirectory); } - for (String directory : directories) { - final List> tasks = new ArrayList<>(); - ExecutorService es = Executors.newFixedThreadPool(10); - for (int i = 0; i < filesPerDirectory; i++) { - final Path fileName = new Path("/" + directory + "/test" + i); - tasks.add(es.submit(() -> { - touch(fileName); - return null; - })); - } - for (Future task : tasks) { - task.get(); - } - es.shutdownNow(); + } + + private void populateDirWithFiles(Path directory, int numFiles) + throws ExecutionException, InterruptedException { + final List> tasks = new ArrayList<>(); + ExecutorService es = Executors.newFixedThreadPool(10); + for (int i = 0; i < numFiles; i++) { + final Path fileName = new Path(directory + "/test" + i); + tasks.add(es.submit(() -> { + touch(fileName); + return null; + })); + } + for (Future task : tasks) { + task.get(); } + es.shutdownNow(); } } From 30cf1957cdc4e3e4ef6fdbd487f55439e7171ee3 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Wed, 16 Dec 2020 10:42:15 +0530 Subject: [PATCH 05/33] checkstyle fix --- .../fs/azurebfs/AzureBlobFileSystem.java | 7 +++--- .../services/ContentSummaryProcessor.java | 2 +- .../fs/azurebfs/utils/ContentSummary.java | 23 +++++++++++++++---- .../fs/azurebfs/TestGetContentSummary.java | 19 +++++++++------ 4 files changed, 36 insertions(+), 15 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index e09ee809ddddf..e0ca9525db643 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -370,9 +370,10 @@ public boolean delete(final Path f, final boolean recursive) throws IOException public ContentSummary getContentSummary(Path f) throws IOException { org.apache.hadoop.fs.azurebfs.utils.ContentSummary contentSummary = (new ContentSummaryProcessor(abfsStore)).getContentSummary(f); - return new Builder().length(contentSummary.length) - .directoryCount(contentSummary.directoryCount).fileCount(contentSummary.fileCount) - .spaceConsumed(contentSummary.spaceConsumed).build(); + return new Builder().length(contentSummary.getLength()) + .directoryCount(contentSummary.getDirectoryCount()) + .fileCount(contentSummary.getFileCount()) + .spaceConsumed(contentSummary.getSpaceConsumed()).build(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 59c2845ba90a1..df0042392b881 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -79,7 +79,7 @@ private void processFile(FileStatus fileStatus) { this.totalBytes.addAndGet(fileStatus.getLen()); } - private class ThreadProcessor implements Runnable { + private final class ThreadProcessor implements Runnable { private ThreadProcessor() { } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java index dd557614033cb..f433e022ea887 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java @@ -19,10 +19,10 @@ package org.apache.hadoop.fs.azurebfs.utils; public class ContentSummary { - public final long length; - public final long directoryCount; - public final long fileCount; - public final long spaceConsumed; + private final long length; + private final long directoryCount; + private final long fileCount; + private final long spaceConsumed; public ContentSummary(long length, long directoryCount, long fileCount, long spaceConsumed) { @@ -31,4 +31,19 @@ public ContentSummary(long length, long directoryCount, long fileCount, this.fileCount = fileCount; this.spaceConsumed = spaceConsumed; } + + public long getLength() { + return length; + } + + public long getDirectoryCount() { + return directoryCount; + } + + public long getFileCount() { + return fileCount; + } + public long getSpaceConsumed() { + return spaceConsumed; + } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java index 82a50fd8a95dd..b002165071be4 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java @@ -42,9 +42,12 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { "testFolder/testFolder2/testFolder5", "testFolder/testFolder3/testFolder6", "testFolder/testFolder3/testFolder7"}; - private final int filesPerDirectory = 2; private final AzureBlobFileSystem fs = createFileSystem(); - private final byte[] b = new byte[20]; + private final int testBufferSize = 20; + private final int filesPerDirectory = 2; + private final int numFilesForListMaxTest = + DEFAULT_AZURE_LIST_MAX_RESULTS + 100; + private final byte[] b = new byte[testBufferSize]; public TestGetContentSummary() throws Exception { createDirectoryStructure(); @@ -65,7 +68,7 @@ public void testFileContentSummary() throws IOException { out.write(b); out.close(); ContentSummary contentSummary = fs.getContentSummary(filePath); - checkContentSummary(contentSummary, 0, 1, 20); + checkContentSummary(contentSummary, 0, 1, testBufferSize); } @Test @@ -87,7 +90,7 @@ public void testIntermediateDirWithFilesOnly() throws IOException { } ContentSummary contentSummary = fs.getContentSummary(new Path(dirPath)); checkContentSummary(contentSummary, 0, filesPerDirectory, - 20 * filesPerDirectory); + testBufferSize * filesPerDirectory); } @Test @@ -105,7 +108,8 @@ public void testIntermediateDirWithFilesAndSubdirs() throws IOException { out.close(); } ContentSummary contentSummary = fs.getContentSummary(dirPath); - checkContentSummary(contentSummary, 2, 3 * filesPerDirectory, 20 * 2 * 2); + checkContentSummary(contentSummary, 2, 3 * filesPerDirectory, + testBufferSize * 2 * 2); } @Test @@ -121,13 +125,14 @@ public void testDirOverListMaxResultsItems() throws IOException, ExecutionException, InterruptedException { Path pathToDir = new Path("/testFolder/testFolder2/maxListDir"); fs.mkdirs(pathToDir); - populateDirWithFiles(pathToDir, DEFAULT_AZURE_LIST_MAX_RESULTS + 100); + populateDirWithFiles(pathToDir, numFilesForListMaxTest); FSDataOutputStream out = fs.append(new Path(pathToDir + "/test0")); out.write(b); out.close(); checkContentSummary( fs.getContentSummary(new Path("/testFolder" + "/testFolder2")), 3, - DEFAULT_AZURE_LIST_MAX_RESULTS + 100 + filesPerDirectory * 3, 20); + numFilesForListMaxTest + filesPerDirectory * 3, + testBufferSize); } private void checkContentSummary(ContentSummary contentSummary, From 95d13965f191ce7c6c4876229b430f5ff8096244 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 17 Dec 2020 12:22:44 +0530 Subject: [PATCH 06/33] linkedBlockingQ + junit test fix --- .../fs/azurebfs/AzureBlobFileSystem.java | 8 +- .../services/ContentSummaryProcessor.java | 31 ++--- .../fs/azurebfs/services/ProcessingQueue.java | 80 ------------- .../fs/azurebfs/TestGetContentSummary.java | 110 +++++++++--------- 4 files changed, 75 insertions(+), 154 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index e0ca9525db643..8c8f18f2a2f7f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -368,8 +368,12 @@ 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 = - (new ContentSummaryProcessor(abfsStore)).getContentSummary(f); + org.apache.hadoop.fs.azurebfs.utils.ContentSummary contentSummary = null; + try { + contentSummary = (new ContentSummaryProcessor(abfsStore)).getContentSummary(f); + } catch (InterruptedException e) { + e.printStackTrace(); + } return new Builder().length(contentSummary.getLength()) .directoryCount(contentSummary.getDirectoryCount()) .fileCount(contentSummary.getFileCount()) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index df0042392b881..ed716d74fec04 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -24,13 +24,16 @@ import org.apache.hadoop.fs.azurebfs.utils.ContentSummary; import java.io.IOException; +import java.util.concurrent.LinkedBlockingDeque; +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 ProcessingQueue queue = new ProcessingQueue<>(); + private final LinkedBlockingDeque queue = + new LinkedBlockingDeque<>(); private final AzureBlobFileSystemStore abfsStore; private static final int NUM_THREADS = 16; @@ -38,9 +41,10 @@ public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { this.abfsStore = abfsStore; } - public ContentSummary getContentSummary(Path path) throws IOException { + public ContentSummary getContentSummary(Path path) + throws IOException, InterruptedException { processDirectoryTree(path); - Thread[] threads = new Thread[16]; + Thread[] threads = new Thread[NUM_THREADS]; for (int i = 0; i < NUM_THREADS; ++i) { threads[i] = new Thread(new ContentSummaryProcessor.ThreadProcessor()); @@ -58,12 +62,13 @@ public ContentSummary getContentSummary(Path path) throws IOException { fileCount.get(), totalBytes.get()); } - private void processDirectoryTree(Path path) throws IOException { + private void processDirectoryTree(Path path) + throws IOException, InterruptedException { FileStatus[] fileStatuses = abfsStore.listStatus(path); for (FileStatus fileStatus : fileStatuses) { if (fileStatus.isDirectory()) { this.processDirectory(); - this.queue.add(fileStatus); + this.queue.put(fileStatus); } else { this.processFile(fileStatus); } @@ -86,16 +91,14 @@ private ThreadProcessor() { public void run() { try { FileStatus fileStatus; - while ((fileStatus = ContentSummaryProcessor.this.queue.poll()) - != null) { - if (fileStatus.isDirectory()) { - ContentSummaryProcessor.this - .processDirectoryTree(fileStatus.getPath()); - } - ContentSummaryProcessor.this.queue.unregister(); + fileStatus = queue.poll(3, TimeUnit.SECONDS); + if (fileStatus == null) + return; + if (fileStatus.isDirectory()) { + processDirectoryTree(fileStatus.getPath()); } - } catch (IOException e) { - throw new RuntimeException("IOException processing Directory tree", e); + } catch (InterruptedException | IOException interruptedException) { + interruptedException.printStackTrace(); } } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java deleted file mode 100644 index 3e7d5ae07a24d..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java +++ /dev/null @@ -1,80 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import java.util.LinkedList; -import java.util.Queue; - -public class ProcessingQueue { - - private final Queue internalQueue = new LinkedList<>(); - private int processorCount = 0; - - ProcessingQueue() { - } - - public synchronized void add(T item) { - if (item == null) { - throw new IllegalArgumentException("Cannot put null into queue"); - } else { - this.internalQueue.add(item); - this.notifyAll(); - } - } - - public synchronized T poll() { - while (true) { - try { - if (this.isQueueEmpty() && !this.done()) { - this.wait(); - continue; - } - if (!this.isQueueEmpty()) { - ++this.processorCount; - return this.internalQueue.poll(); - } - return null; - } catch (InterruptedException var2) { - Thread.currentThread().interrupt(); - } - return null; - } - } - - public synchronized void unregister() { - --this.processorCount; - if (this.processorCount < 0) { - throw new IllegalStateException( - "too many unregister()'s. processorCount is now " - + this.processorCount); - } else { - if (this.done()) { - this.notifyAll(); - } - } - } - - private boolean done() { - return this.processorCount == 0 && this.isQueueEmpty(); - } - - private boolean isQueueEmpty() { - return this.internalQueue.peek() == null; - } -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java index b002165071be4..f26f789b05285 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java @@ -36,12 +36,30 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS; 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"}; + + 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/zFolder"}; + //thread poll should not get interrupted before zFolder is put in queue + + 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; @@ -55,84 +73,49 @@ public TestGetContentSummary() throws Exception { } @Test - public void testFilesystemRoot() throws IOException { + public void testFilesystemRoot() + throws IOException { + int fileCount = + (directories.length - 2) * filesPerDirectory + numFilesForListMaxTest; ContentSummary contentSummary = fs.getContentSummary(new Path("/")); - checkContentSummary(contentSummary, directories.length, - directories.length * filesPerDirectory, 0); + checkContentSummary(contentSummary, directories.length, fileCount, + dirsWithNonEmptyFiles.length * filesPerDirectory * testBufferSize); } @Test public void testFileContentSummary() throws IOException { - Path filePath = new Path("/testFolderII/testFile"); - FSDataOutputStream out = fs.create(filePath); - out.write(b); - out.close(); - ContentSummary contentSummary = fs.getContentSummary(filePath); + ContentSummary contentSummary = fs.getContentSummary(pathToFile); checkContentSummary(contentSummary, 0, 1, testBufferSize); } @Test public void testLeafDir() throws IOException { - Path pathToLeafDir = new Path( - "/testFolder/testFolder2/testFolder4" + "/leafDir"); - fs.mkdirs(pathToLeafDir); ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir); checkContentSummary(contentSummary, 0, 0, 0); } @Test public void testIntermediateDirWithFilesOnly() throws IOException { - String dirPath = "/testFolder/testFolder3/testFolder6"; - for (int i = 0; i < filesPerDirectory; i++) { - FSDataOutputStream out = fs.append(new Path(dirPath + "/test" + i)); - out.write(b); - out.close(); - } - ContentSummary contentSummary = fs.getContentSummary(new Path(dirPath)); + ContentSummary contentSummary = + fs.getContentSummary(pathToIntermediateDirWithFilesOnly); checkContentSummary(contentSummary, 0, filesPerDirectory, testBufferSize * filesPerDirectory); } @Test public void testIntermediateDirWithFilesAndSubdirs() throws IOException { - Path dirPath = new Path("/testFolder/testFolder3"); - for (int i = 0; i < filesPerDirectory; i++) { - FSDataOutputStream out = fs.append(new Path(dirPath + "/test" + i)); - out.write(b); - out.close(); - } - Path dir2Path = new Path("/testFolder/testFolder3/testFolder6"); - for (int i = 0; i < filesPerDirectory; i++) { - FSDataOutputStream out = fs.append(new Path(dir2Path + "/test" + i)); - out.write(b); - out.close(); - } - ContentSummary contentSummary = fs.getContentSummary(dirPath); - checkContentSummary(contentSummary, 2, 3 * filesPerDirectory, - testBufferSize * 2 * 2); - } - - @Test - public void testEmptyDir() throws IOException { - Path pathToEmptyDir = new Path("/testFolder/emptyDir"); - fs.mkdirs(pathToEmptyDir); - ContentSummary contentSummary = fs.getContentSummary(pathToEmptyDir); - checkContentSummary(contentSummary, 0, 0, 0); + ContentSummary contentSummary = + fs.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs); + checkContentSummary(contentSummary, 3, 3 * filesPerDirectory, + testBufferSize * filesPerDirectory); } @Test public void testDirOverListMaxResultsItems() throws IOException, ExecutionException, InterruptedException { - Path pathToDir = new Path("/testFolder/testFolder2/maxListDir"); - fs.mkdirs(pathToDir); - populateDirWithFiles(pathToDir, numFilesForListMaxTest); - FSDataOutputStream out = fs.append(new Path(pathToDir + "/test0")); - out.write(b); - out.close(); checkContentSummary( - fs.getContentSummary(new Path("/testFolder" + "/testFolder2")), 3, - numFilesForListMaxTest + filesPerDirectory * 3, - testBufferSize); + fs.getContentSummary(pathToListMaxDir), 1, + numFilesForListMaxTest + filesPerDirectory, 0); } private void checkContentSummary(ContentSummary contentSummary, @@ -150,10 +133,21 @@ private void checkContentSummary(ContentSummary contentSummary, private void createDirectoryStructure() throws IOException, ExecutionException, InterruptedException { for (String directory : directories) { - Path dirPath = new Path("/" + directory); + Path dirPath = new Path(directory); fs.mkdirs(dirPath); - populateDirWithFiles(dirPath, filesPerDirectory); + if (!(dirPath.equals(pathToLeafDir) || dirPath + .equals(pathToListMaxDir))) { + populateDirWithFiles(dirPath, filesPerDirectory); + } + } + for (String dir : dirsWithNonEmptyFiles) { + for (int i = 0; i < filesPerDirectory; i++) { + FSDataOutputStream out = fs.append(new Path(dir + "/test" + i)); + out.write(b); + out.close(); + } } + populateDirWithFiles(pathToListMaxDir, numFilesForListMaxTest); } private void populateDirWithFiles(Path directory, int numFiles) From 03d342ce5053b738b9936fdc31fd40ddc4b1f3b8 Mon Sep 17 00:00:00 2001 From: sumangala-patki <70206833+sumangala-patki@users.noreply.github.com> Date: Thu, 17 Dec 2020 12:26:05 +0530 Subject: [PATCH 07/33] linkedBlockingQ + junit test fix (#5) --- .../fs/azurebfs/AzureBlobFileSystem.java | 8 +- .../services/ContentSummaryProcessor.java | 31 ++--- .../fs/azurebfs/services/ProcessingQueue.java | 80 ------------- .../fs/azurebfs/TestGetContentSummary.java | 110 +++++++++--------- 4 files changed, 75 insertions(+), 154 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index e0ca9525db643..8c8f18f2a2f7f 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -368,8 +368,12 @@ 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 = - (new ContentSummaryProcessor(abfsStore)).getContentSummary(f); + org.apache.hadoop.fs.azurebfs.utils.ContentSummary contentSummary = null; + try { + contentSummary = (new ContentSummaryProcessor(abfsStore)).getContentSummary(f); + } catch (InterruptedException e) { + e.printStackTrace(); + } return new Builder().length(contentSummary.getLength()) .directoryCount(contentSummary.getDirectoryCount()) .fileCount(contentSummary.getFileCount()) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index df0042392b881..ed716d74fec04 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -24,13 +24,16 @@ import org.apache.hadoop.fs.azurebfs.utils.ContentSummary; import java.io.IOException; +import java.util.concurrent.LinkedBlockingDeque; +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 ProcessingQueue queue = new ProcessingQueue<>(); + private final LinkedBlockingDeque queue = + new LinkedBlockingDeque<>(); private final AzureBlobFileSystemStore abfsStore; private static final int NUM_THREADS = 16; @@ -38,9 +41,10 @@ public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { this.abfsStore = abfsStore; } - public ContentSummary getContentSummary(Path path) throws IOException { + public ContentSummary getContentSummary(Path path) + throws IOException, InterruptedException { processDirectoryTree(path); - Thread[] threads = new Thread[16]; + Thread[] threads = new Thread[NUM_THREADS]; for (int i = 0; i < NUM_THREADS; ++i) { threads[i] = new Thread(new ContentSummaryProcessor.ThreadProcessor()); @@ -58,12 +62,13 @@ public ContentSummary getContentSummary(Path path) throws IOException { fileCount.get(), totalBytes.get()); } - private void processDirectoryTree(Path path) throws IOException { + private void processDirectoryTree(Path path) + throws IOException, InterruptedException { FileStatus[] fileStatuses = abfsStore.listStatus(path); for (FileStatus fileStatus : fileStatuses) { if (fileStatus.isDirectory()) { this.processDirectory(); - this.queue.add(fileStatus); + this.queue.put(fileStatus); } else { this.processFile(fileStatus); } @@ -86,16 +91,14 @@ private ThreadProcessor() { public void run() { try { FileStatus fileStatus; - while ((fileStatus = ContentSummaryProcessor.this.queue.poll()) - != null) { - if (fileStatus.isDirectory()) { - ContentSummaryProcessor.this - .processDirectoryTree(fileStatus.getPath()); - } - ContentSummaryProcessor.this.queue.unregister(); + fileStatus = queue.poll(3, TimeUnit.SECONDS); + if (fileStatus == null) + return; + if (fileStatus.isDirectory()) { + processDirectoryTree(fileStatus.getPath()); } - } catch (IOException e) { - throw new RuntimeException("IOException processing Directory tree", e); + } catch (InterruptedException | IOException interruptedException) { + interruptedException.printStackTrace(); } } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java deleted file mode 100644 index 3e7d5ae07a24d..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ProcessingQueue.java +++ /dev/null @@ -1,80 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.services; - -import java.util.LinkedList; -import java.util.Queue; - -public class ProcessingQueue { - - private final Queue internalQueue = new LinkedList<>(); - private int processorCount = 0; - - ProcessingQueue() { - } - - public synchronized void add(T item) { - if (item == null) { - throw new IllegalArgumentException("Cannot put null into queue"); - } else { - this.internalQueue.add(item); - this.notifyAll(); - } - } - - public synchronized T poll() { - while (true) { - try { - if (this.isQueueEmpty() && !this.done()) { - this.wait(); - continue; - } - if (!this.isQueueEmpty()) { - ++this.processorCount; - return this.internalQueue.poll(); - } - return null; - } catch (InterruptedException var2) { - Thread.currentThread().interrupt(); - } - return null; - } - } - - public synchronized void unregister() { - --this.processorCount; - if (this.processorCount < 0) { - throw new IllegalStateException( - "too many unregister()'s. processorCount is now " - + this.processorCount); - } else { - if (this.done()) { - this.notifyAll(); - } - } - } - - private boolean done() { - return this.processorCount == 0 && this.isQueueEmpty(); - } - - private boolean isQueueEmpty() { - return this.internalQueue.peek() == null; - } -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java index b002165071be4..f26f789b05285 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java @@ -36,12 +36,30 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS; 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"}; + + 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/zFolder"}; + //thread poll should not get interrupted before zFolder is put in queue + + 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; @@ -55,84 +73,49 @@ public TestGetContentSummary() throws Exception { } @Test - public void testFilesystemRoot() throws IOException { + public void testFilesystemRoot() + throws IOException { + int fileCount = + (directories.length - 2) * filesPerDirectory + numFilesForListMaxTest; ContentSummary contentSummary = fs.getContentSummary(new Path("/")); - checkContentSummary(contentSummary, directories.length, - directories.length * filesPerDirectory, 0); + checkContentSummary(contentSummary, directories.length, fileCount, + dirsWithNonEmptyFiles.length * filesPerDirectory * testBufferSize); } @Test public void testFileContentSummary() throws IOException { - Path filePath = new Path("/testFolderII/testFile"); - FSDataOutputStream out = fs.create(filePath); - out.write(b); - out.close(); - ContentSummary contentSummary = fs.getContentSummary(filePath); + ContentSummary contentSummary = fs.getContentSummary(pathToFile); checkContentSummary(contentSummary, 0, 1, testBufferSize); } @Test public void testLeafDir() throws IOException { - Path pathToLeafDir = new Path( - "/testFolder/testFolder2/testFolder4" + "/leafDir"); - fs.mkdirs(pathToLeafDir); ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir); checkContentSummary(contentSummary, 0, 0, 0); } @Test public void testIntermediateDirWithFilesOnly() throws IOException { - String dirPath = "/testFolder/testFolder3/testFolder6"; - for (int i = 0; i < filesPerDirectory; i++) { - FSDataOutputStream out = fs.append(new Path(dirPath + "/test" + i)); - out.write(b); - out.close(); - } - ContentSummary contentSummary = fs.getContentSummary(new Path(dirPath)); + ContentSummary contentSummary = + fs.getContentSummary(pathToIntermediateDirWithFilesOnly); checkContentSummary(contentSummary, 0, filesPerDirectory, testBufferSize * filesPerDirectory); } @Test public void testIntermediateDirWithFilesAndSubdirs() throws IOException { - Path dirPath = new Path("/testFolder/testFolder3"); - for (int i = 0; i < filesPerDirectory; i++) { - FSDataOutputStream out = fs.append(new Path(dirPath + "/test" + i)); - out.write(b); - out.close(); - } - Path dir2Path = new Path("/testFolder/testFolder3/testFolder6"); - for (int i = 0; i < filesPerDirectory; i++) { - FSDataOutputStream out = fs.append(new Path(dir2Path + "/test" + i)); - out.write(b); - out.close(); - } - ContentSummary contentSummary = fs.getContentSummary(dirPath); - checkContentSummary(contentSummary, 2, 3 * filesPerDirectory, - testBufferSize * 2 * 2); - } - - @Test - public void testEmptyDir() throws IOException { - Path pathToEmptyDir = new Path("/testFolder/emptyDir"); - fs.mkdirs(pathToEmptyDir); - ContentSummary contentSummary = fs.getContentSummary(pathToEmptyDir); - checkContentSummary(contentSummary, 0, 0, 0); + ContentSummary contentSummary = + fs.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs); + checkContentSummary(contentSummary, 3, 3 * filesPerDirectory, + testBufferSize * filesPerDirectory); } @Test public void testDirOverListMaxResultsItems() throws IOException, ExecutionException, InterruptedException { - Path pathToDir = new Path("/testFolder/testFolder2/maxListDir"); - fs.mkdirs(pathToDir); - populateDirWithFiles(pathToDir, numFilesForListMaxTest); - FSDataOutputStream out = fs.append(new Path(pathToDir + "/test0")); - out.write(b); - out.close(); checkContentSummary( - fs.getContentSummary(new Path("/testFolder" + "/testFolder2")), 3, - numFilesForListMaxTest + filesPerDirectory * 3, - testBufferSize); + fs.getContentSummary(pathToListMaxDir), 1, + numFilesForListMaxTest + filesPerDirectory, 0); } private void checkContentSummary(ContentSummary contentSummary, @@ -150,10 +133,21 @@ private void checkContentSummary(ContentSummary contentSummary, private void createDirectoryStructure() throws IOException, ExecutionException, InterruptedException { for (String directory : directories) { - Path dirPath = new Path("/" + directory); + Path dirPath = new Path(directory); fs.mkdirs(dirPath); - populateDirWithFiles(dirPath, filesPerDirectory); + if (!(dirPath.equals(pathToLeafDir) || dirPath + .equals(pathToListMaxDir))) { + populateDirWithFiles(dirPath, filesPerDirectory); + } + } + for (String dir : dirsWithNonEmptyFiles) { + for (int i = 0; i < filesPerDirectory; i++) { + FSDataOutputStream out = fs.append(new Path(dir + "/test" + i)); + out.write(b); + out.close(); + } } + populateDirWithFiles(pathToListMaxDir, numFilesForListMaxTest); } private void populateDirWithFiles(Path directory, int numFiles) From bb55b14694913c62bb2ae69992bf848cd289d340 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 22 Dec 2020 23:53:21 +0530 Subject: [PATCH 08/33] using executors --- .../services/ContentSummaryProcessor.java | 73 +++++++++---------- .../fs/azurebfs/TestGetContentSummary.java | 3 +- 2 files changed, 34 insertions(+), 42 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index ed716d74fec04..15d472d9fcf4a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -24,7 +24,11 @@ import org.apache.hadoop.fs.azurebfs.utils.ContentSummary; import java.io.IOException; -import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +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; @@ -32,10 +36,12 @@ 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 LinkedBlockingDeque queue = - new LinkedBlockingDeque<>(); + private final LinkedBlockingQueue 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; @@ -44,20 +50,11 @@ public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { public ContentSummary getContentSummary(Path path) throws IOException, InterruptedException { processDirectoryTree(path); - Thread[] threads = new Thread[NUM_THREADS]; - for (int i = 0; i < NUM_THREADS; ++i) { - threads[i] = new Thread(new ContentSummaryProcessor.ThreadProcessor()); - threads[i].start(); - } - - for (Thread t : threads) { - try { - t.join(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - } + while (((ThreadPoolExecutor) executorService).getActiveCount() > 0) { + Thread.sleep(100); } + executorService.shutdown(); return new ContentSummary(totalBytes.get(), directoryCount.get(), fileCount.get(), totalBytes.get()); } @@ -65,41 +62,37 @@ public ContentSummary getContentSummary(Path path) private void processDirectoryTree(Path path) throws IOException, InterruptedException { FileStatus[] fileStatuses = abfsStore.listStatus(path); + for (FileStatus fileStatus : fileStatuses) { if (fileStatus.isDirectory()) { - this.processDirectory(); - this.queue.put(fileStatus); + 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(); + } + }); + } + } } else { - this.processFile(fileStatus); + processFile(fileStatus); } } } private void processDirectory() { - this.directoryCount.incrementAndGet(); + directoryCount.incrementAndGet(); } private void processFile(FileStatus fileStatus) { - this.fileCount.incrementAndGet(); - this.totalBytes.addAndGet(fileStatus.getLen()); - } - - private final class ThreadProcessor implements Runnable { - private ThreadProcessor() { - } - - public void run() { - try { - FileStatus fileStatus; - fileStatus = queue.poll(3, TimeUnit.SECONDS); - if (fileStatus == null) - return; - if (fileStatus.isDirectory()) { - processDirectoryTree(fileStatus.getPath()); - } - } catch (InterruptedException | IOException interruptedException) { - interruptedException.printStackTrace(); - } - } + fileCount.incrementAndGet(); + totalBytes.addAndGet(fileStatus.getLen()); } } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java index f26f789b05285..f4d4b1d17dab3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java @@ -47,7 +47,6 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { "/testFolder/testFolder3/testFolder6/leafDir", "/testFolderII/listMaxDir", "/testFolderII/listMaxDir/zFolder"}; - //thread poll should not get interrupted before zFolder is put in queue private final Path pathToFile = new Path("/testFolder/test1");; private final Path pathToListMaxDir = new Path("/testFolderII/listMaxDir"); @@ -64,7 +63,7 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { private final int testBufferSize = 20; private final int filesPerDirectory = 2; private final int numFilesForListMaxTest = - DEFAULT_AZURE_LIST_MAX_RESULTS + 100; + DEFAULT_AZURE_LIST_MAX_RESULTS + 10; private final byte[] b = new byte[testBufferSize]; public TestGetContentSummary() throws Exception { From 06609da46d1eaea4f31f96426f461317d2c0456d Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 24 Dec 2020 13:26:17 +0530 Subject: [PATCH 09/33] run()->call(), terminate condition, add invalid path test --- .../fs/azurebfs/AzureBlobFileSystem.java | 24 ++++++----- .../services/ContentSummaryProcessor.java | 42 ++++++++++++------- ...ntSummary.java => ABFSContentSummary.java} | 4 +- .../fs/azurebfs/TestGetContentSummary.java | 22 +++++++--- 4 files changed, 58 insertions(+), 34 deletions(-) rename hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/{ContentSummary.java => ABFSContentSummary.java} (92%) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 8c8f18f2a2f7f..0ac0304b16c30 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -37,9 +37,6 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; -import org.apache.hadoop.fs.ContentSummary; -import org.apache.hadoop.fs.ContentSummary.Builder; -import org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; import org.slf4j.Logger; @@ -52,6 +49,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.CommonPathCapabilities; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.ContentSummary.Builder; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -73,6 +72,8 @@ import org.apache.hadoop.fs.azurebfs.contracts.services.AzureServiceErrorCode; import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.azurebfs.services.AbfsCounters; +import org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor; +import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary; import org.apache.hadoop.fs.permission.AclEntry; import org.apache.hadoop.fs.permission.AclStatus; import org.apache.hadoop.fs.permission.FsAction; @@ -367,17 +368,18 @@ 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; + public ContentSummary getContentSummary(Path path) throws IOException { try { - contentSummary = (new ContentSummaryProcessor(abfsStore)).getContentSummary(f); + 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) { - e.printStackTrace(); + LOG.debug(e.toString()); + throw new IOException(e.getMessage()); } - return new Builder().length(contentSummary.getLength()) - .directoryCount(contentSummary.getDirectoryCount()) - .fileCount(contentSummary.getFileCount()) - .spaceConsumed(contentSummary.getSpaceConsumed()).build(); } @Override diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index d4a02b2f5b8a2..dd66e4127c905 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -18,43 +18,49 @@ 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.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 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<>()); - //cached thread pool with custom max threads to avoid overloading public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { this.abfsStore = abfsStore; } - public ContentSummary getContentSummary(Path path) + public ABFSContentSummary getContentSummary(Path path) throws IOException, InterruptedException { processDirectoryTree(path); - while (((ThreadPoolExecutor) executorService).getActiveCount() > 0) { - Thread.sleep(100); + while(!queue.isEmpty() || NUM_TASKS.get() > 0) { + Thread.sleep(10); } executorService.shutdown(); - return new ContentSummary(totalBytes.get(), directoryCount.get(), + return new ABFSContentSummary(totalBytes.get(), directoryCount.get(), fileCount.get(), totalBytes.get()); } @@ -67,20 +73,26 @@ private void processDirectoryTree(Path path) queue.put(fileStatus); processDirectory(); synchronized (this) { - if (!queue.isEmpty()) { + if (!queue.isEmpty() && NUM_TASKS.get() < NUM_THREADS) { + NUM_TASKS.incrementAndGet(); executorService.submit(() -> { try { FileStatus fileStatus1; - while ((fileStatus1 = queue.poll(100, TimeUnit.MILLISECONDS)) != null) { + while ((fileStatus1 = queue.poll(100, TimeUnit.MILLISECONDS)) + != null) { processDirectoryTree(fileStatus1.getPath()); } - } catch (InterruptedException | IOException e) { - e.printStackTrace(); + NUM_TASKS.decrementAndGet(); + } catch (IOException | InterruptedException e) { + NUM_TASKS.decrementAndGet(); + LOG.debug(e.toString()); + throw new IOException(e.getMessage()); } + return null; }); } } - } else { + }else { processFile(fileStatus); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ABFSContentSummary.java similarity index 92% rename from hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java rename to hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ABFSContentSummary.java index f433e022ea887..4ac505569291e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ABFSContentSummary.java @@ -18,13 +18,13 @@ package org.apache.hadoop.fs.azurebfs.utils; -public class ContentSummary { +public class ABFSContentSummary { private final long length; private final long directoryCount; private final long fileCount; private final long spaceConsumed; - public ContentSummary(long length, long directoryCount, long fileCount, + public ABFSContentSummary(long length, long directoryCount, long fileCount, long spaceConsumed) { this.length = length; this.directoryCount = directoryCount; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java index f4d4b1d17dab3..72f6d85c38e20 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java @@ -18,12 +18,6 @@ package org.apache.hadoop.fs.azurebfs; -import org.apache.hadoop.fs.ContentSummary; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.Path; -import org.assertj.core.api.Assertions; -import org.junit.Test; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -33,7 +27,15 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +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 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 { @@ -117,6 +119,14 @@ public void testDirOverListMaxResultsItems() numFilesForListMaxTest + filesPerDirectory, 0); } + @Test + public void testInvalidPath() throws Exception { + intercept(IOException.class, () -> fs.getContentSummary(new Path( + "/nonExistentPath"))); + intercept(IOException.class, () -> fs.getContentSummary(new Path( + "testFolder/IntermediateNonExistentPath"))); + } + private void checkContentSummary(ContentSummary contentSummary, long directoryCount, long fileCount, long byteCount) { Assertions.assertThat(contentSummary.getDirectoryCount()) From 1433c858ce1280100ca23a39b3a02e77e4d1ad11 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 24 Dec 2020 16:32:32 +0530 Subject: [PATCH 10/33] pr revw + checkstyle --- .../services/ContentSummaryProcessor.java | 25 ++++++++++--------- 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index dd66e4127c905..b9e125c2459c1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -39,14 +39,15 @@ 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 queue = new LinkedBlockingQueue<>(); - private final Logger LOG = + private static 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<>()); + private static final int POLL_TIMEOUT = 100; public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { this.abfsStore = abfsStore; @@ -56,7 +57,7 @@ public ABFSContentSummary getContentSummary(Path path) throws IOException, InterruptedException { processDirectoryTree(path); - while(!queue.isEmpty() || NUM_TASKS.get() > 0) { + while (!queue.isEmpty() || numTasks.get() > 0) { Thread.sleep(10); } executorService.shutdown(); @@ -73,26 +74,26 @@ private void processDirectoryTree(Path path) queue.put(fileStatus); processDirectory(); synchronized (this) { - if (!queue.isEmpty() && NUM_TASKS.get() < NUM_THREADS) { - NUM_TASKS.incrementAndGet(); + if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) { + numTasks.incrementAndGet(); executorService.submit(() -> { try { FileStatus fileStatus1; - while ((fileStatus1 = queue.poll(100, TimeUnit.MILLISECONDS)) + while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS)) != null) { processDirectoryTree(fileStatus1.getPath()); } - NUM_TASKS.decrementAndGet(); } catch (IOException | InterruptedException e) { - NUM_TASKS.decrementAndGet(); LOG.debug(e.toString()); throw new IOException(e.getMessage()); + } finally { + numTasks.decrementAndGet(); } return null; }); } } - }else { + } else { processFile(fileStatus); } } From d747f06ac5c94e56b368e3f49cf0121d2f6444b1 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Thu, 24 Dec 2020 22:41:56 +0530 Subject: [PATCH 11/33] findbugs use future returned --- .../services/ContentSummaryProcessor.java | 39 ++++++++++++------- 1 file changed, 26 insertions(+), 13 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index b9e125c2459c1..ee1f604e64cb5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -19,11 +19,17 @@ 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; @@ -44,6 +50,8 @@ public class ContentSummaryProcessor { private final ExecutorService executorService = new ThreadPoolExecutor(1, NUM_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>()); private final LinkedBlockingQueue queue = new LinkedBlockingQueue<>(); + private final Set> futures = + Collections.newSetFromMap(new ConcurrentHashMap<>()); private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class); private static final int NUM_THREADS = 16; @@ -58,8 +66,18 @@ public ABFSContentSummary getContentSummary(Path path) processDirectoryTree(path); while (!queue.isEmpty() || numTasks.get() > 0) { - Thread.sleep(10); + for (Future 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()); @@ -76,21 +94,16 @@ private void processDirectoryTree(Path path) synchronized (this) { if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) { numTasks.incrementAndGet(); - executorService.submit(() -> { - try { - FileStatus fileStatus1; - while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS)) - != null) { - processDirectoryTree(fileStatus1.getPath()); - } - } catch (IOException | InterruptedException e) { - LOG.debug(e.toString()); - throw new IOException(e.getMessage()); - } finally { - numTasks.decrementAndGet(); + Future future = executorService.submit(() -> { + FileStatus fileStatus1; + while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS)) + != null) { + processDirectoryTree(fileStatus1.getPath()); } + numTasks.decrementAndGet(); return null; }); + futures.add(future); } } } else { From be2daf0f7acd86857a2aac8253410f88f1e2c18c Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 5 Jan 2021 11:46:14 +0530 Subject: [PATCH 12/33] completion service + temp concurrency tests --- .../services/ContentSummaryProcessor.java | 78 +++++++++++++++---- .../hadoop/fs/azurebfs/utils/Listener.java | 21 +++++ .../{ => services}/TestGetContentSummary.java | 28 ++++++- .../utils/ExecutorServiceTestUtils.java | 66 ++++++++++++++++ 4 files changed, 173 insertions(+), 20 deletions(-) create mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/{ => services}/TestGetContentSummary.java (84%) create mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ExecutorServiceTestUtils.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index ee1f604e64cb5..d2dfeeafecf3c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -21,18 +21,21 @@ 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.TimeoutException; 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; @@ -41,21 +44,24 @@ import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary; -public class ContentSummaryProcessor { +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 completionService = new ExecutorCompletionService<>( + executorService); private final LinkedBlockingQueue queue = new LinkedBlockingQueue<>(); - private final Set> futures = + private final Set> 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; + private Listener listener = null; public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { this.abfsStore = abfsStore; @@ -65,20 +71,33 @@ public ABFSContentSummary getContentSummary(Path path) throws IOException, InterruptedException { processDirectoryTree(path); - while (!queue.isEmpty() || numTasks.get() > 0) { - for (Future 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); + 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()); } - executorService.shutdown(); return new ABFSContentSummary(totalBytes.get(), directoryCount.get(), fileCount.get(), totalBytes.get()); } @@ -86,6 +105,12 @@ public ABFSContentSummary getContentSummary(Path path) 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()) { @@ -94,13 +119,20 @@ private void processDirectoryTree(Path path) synchronized (this) { if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) { numTasks.incrementAndGet(); - Future future = executorService.submit(() -> { + Future future = completionService.submit(() -> { FileStatus fileStatus1; while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS)) != null) { processDirectoryTree(fileStatus1.getPath()); } - numTasks.decrementAndGet(); + if (listener != null) { + synchronized (this) { + if (!listener.isInterrupted() && listener.shouldInterrupt()) { + listener.setInterrupted(); + throw new InterruptedException(); + } + } + } return null; }); futures.add(future); @@ -120,4 +152,18 @@ private void processFile(FileStatus fileStatus) { fileCount.incrementAndGet(); totalBytes.addAndGet(fileStatus.getLen()); } + + @Override + public void close() { + while (completionService.poll() != null); + executorService.shutdown(); + if (!executorService.isTerminated()) { + executorService.shutdownNow(); + } + } + + @VisibleForTesting + void registerListener(Listener listener) { + this.listener = listener; + } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java new file mode 100644 index 0000000000000..5136fa6757a87 --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java @@ -0,0 +1,21 @@ +package org.apache.hadoop.fs.azurebfs.utils; + +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicInteger; + +public interface Listener { + void verifyThreadCount(AtomicInteger numTasks, ThreadPoolExecutor executor, + int maxThreads); + + void checkShutdown(int activeCount); + + void checkAllTasksComplete(AtomicInteger numTasks, int activeCount); + + boolean shouldInterrupt(); + + void setInterrupted(); + + boolean isInterrupted(); + + void checkInterrupt(); +} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java similarity index 84% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index 72f6d85c38e20..7df35fe8ee436 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.hadoop.fs.azurebfs; +package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; import java.util.ArrayList; @@ -26,7 +26,12 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; +import org.apache.hadoop.fs.azurebfs.utils.ExecutorServiceTestUtils; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -36,6 +41,10 @@ import static org.apache.hadoop.fs.azurebfs.constants.FileSystemConfigurations.DEFAULT_AZURE_LIST_MAX_RESULTS; import static org.apache.hadoop.test.LambdaTestUtils.intercept; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; public class TestGetContentSummary extends AbstractAbfsIntegrationTest { @@ -64,8 +73,7 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { 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 int numFilesForListMaxTest = DEFAULT_AZURE_LIST_MAX_RESULTS + 10; private final byte[] b = new byte[testBufferSize]; public TestGetContentSummary() throws Exception { @@ -113,7 +121,7 @@ public void testIntermediateDirWithFilesAndSubdirs() throws IOException { @Test public void testDirOverListMaxResultsItems() - throws IOException, ExecutionException, InterruptedException { + throws IOException { checkContentSummary( fs.getContentSummary(pathToListMaxDir), 1, numFilesForListMaxTest + filesPerDirectory, 0); @@ -127,6 +135,18 @@ public void testInvalidPath() throws Exception { "testFolder/IntermediateNonExistentPath"))); } + @Test + public void testExecutorServiceConcurrency() throws Exception { + AzureBlobFileSystemStore mockStore = getAbfsStore(fs); + ContentSummaryProcessor contentSummaryProcessor = + new ContentSummaryProcessor(mockStore); + contentSummaryProcessor.registerListener(new ExecutorServiceTestUtils(false)); + contentSummaryProcessor.getContentSummary(new Path("/")); + contentSummaryProcessor.registerListener(new ExecutorServiceTestUtils(true)); + intercept(IOException.class, () -> + contentSummaryProcessor.getContentSummary(new Path("/"))); + } + private void checkContentSummary(ContentSummary contentSummary, long directoryCount, long fileCount, long byteCount) { Assertions.assertThat(contentSummary.getDirectoryCount()) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ExecutorServiceTestUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ExecutorServiceTestUtils.java new file mode 100644 index 0000000000000..106bc932a0a0e --- /dev/null +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ExecutorServiceTestUtils.java @@ -0,0 +1,66 @@ +package org.apache.hadoop.fs.azurebfs.utils; + +import org.assertj.core.api.Assertions; + +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicInteger; + +public class ExecutorServiceTestUtils implements Listener { + boolean interrupt; + boolean hasInterrupted = false; + AtomicInteger numInterrupts = new AtomicInteger(0); + public ExecutorServiceTestUtils(boolean testWithInterrupt) { + interrupt = testWithInterrupt; + } + @Override + public void verifyThreadCount(AtomicInteger numTasks, ThreadPoolExecutor executor, + int maxThreads) { + int activeThreads = executor.getActiveCount(); + int taskQueueSize = executor.getQueue().size(); + Assertions.assertThat(activeThreads).isLessThanOrEqualTo(maxThreads); + Assertions.assertThat(activeThreads).isGreaterThanOrEqualTo(Math.min(numTasks.get() - 2, maxThreads)) + .isLessThanOrEqualTo(numTasks.get() + 2); + if (taskQueueSize > maxThreads) { + Assertions.assertThat(activeThreads) + .describedAs("all threads should be active") + .isEqualTo(maxThreads); + } + } + + @Override + public void checkShutdown(int activeCount) { + Assertions.assertThat(activeCount) + .describedAs("All tasks should be halted").isEqualTo(0); + } + + @Override + public void checkAllTasksComplete(AtomicInteger numTasks, int activeCount) { + Assertions.assertThat(numTasks.get()) + .describedAs("Number of tasks should be 0") + .isEqualTo(0); + Assertions.assertThat(activeCount) + .describedAs("All threads should be halted").isEqualTo(0); + } + + @Override + public boolean shouldInterrupt() { + return interrupt; + } + + @Override + public void setInterrupted() { + hasInterrupted = true; + } + + @Override + public boolean isInterrupted() { + return hasInterrupted; + } + + @Override + public void checkInterrupt() { + Assertions.assertThat(isInterrupted()) + .describedAs("Thread should have been interrupted") + .isEqualTo(false); + } +} From 96cd2b959a9281c9f452c071416acda401ebca2c Mon Sep 17 00:00:00 2001 From: Sumangala Date: Fri, 8 Jan 2021 01:17:56 +0530 Subject: [PATCH 13/33] pr revw + exec test --- .../fs/azurebfs/AzureBlobFileSystem.java | 7 +- .../services/ContentSummaryProcessor.java | 97 ++++--------------- .../hadoop/fs/azurebfs/utils/Listener.java | 21 ---- .../services/TestGetContentSummary.java | 63 ++++++++---- .../utils/ExecutorServiceTestUtils.java | 66 ------------- 5 files changed, 71 insertions(+), 183 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java delete mode 100644 hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ExecutorServiceTestUtils.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 0ac0304b16c30..ab7f5acbabdec 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -376,9 +376,10 @@ public ContentSummary getContentSummary(Path path) throws IOException { .directoryCount(contentSummary.getDirectoryCount()) .fileCount(contentSummary.getFileCount()) .spaceConsumed(contentSummary.getSpaceConsumed()).build(); - } catch (InterruptedException e) { - LOG.debug(e.toString()); - throw new IOException(e.getMessage()); + } catch (InterruptedException | ExecutionException e) { + LOG.debug((e instanceof InterruptedException || e.getCause() instanceof InterruptedException)? + "Thread interrupted": e.getCause().getMessage()); + throw new IOException(e); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index d2dfeeafecf3c..706dd4cf03666 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -19,22 +19,13 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; +import java.util.ArrayList; 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.*; 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; @@ -44,7 +35,7 @@ import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary; -public class ContentSummaryProcessor implements AutoCloseable { +public class ContentSummaryProcessor { private final AtomicLong fileCount = new AtomicLong(0L); private final AtomicLong directoryCount = new AtomicLong(0L); private final AtomicLong totalBytes = new AtomicLong(0L); @@ -56,47 +47,27 @@ public class ContentSummaryProcessor implements AutoCloseable { private final CompletionService completionService = new ExecutorCompletionService<>( executorService); private final LinkedBlockingQueue queue = new LinkedBlockingQueue<>(); - private final Set> 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 { + throws IOException, ExecutionException, InterruptedException { + processDirectoryTree(path); try { - while (!queue.isEmpty() || numTasks.get() > 0 - || ((ThreadPoolExecutor) executorService).getActiveCount() > 0) { - numTasks.decrementAndGet(); + while (!queue.isEmpty() || numTasks.get() > 0) { completionService.take().get(); - if (listener != null) { - listener.checkInterrupt(); - } + numTasks.decrementAndGet(); } - } catch (ExecutionException e) { - LOG.debug(e.getMessage()); - throw new IOException(e); } finally { + numTasks.decrementAndGet(); 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()); } + executorService.awaitTermination(1, TimeUnit.SECONDS); return new ABFSContentSummary(totalBytes.get(), directoryCount.get(), fileCount.get(), totalBytes.get()); @@ -105,39 +76,12 @@ public ABFSContentSummary getContentSummary(Path path) 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 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); - } - } + conditionalSubmitTaskToExecutor(); } else { processFile(fileStatus); } @@ -153,17 +97,18 @@ private void processFile(FileStatus fileStatus) { totalBytes.addAndGet(fileStatus.getLen()); } - @Override - public void close() { - while (completionService.poll() != null); - executorService.shutdown(); - if (!executorService.isTerminated()) { - executorService.shutdownNow(); + private synchronized void conditionalSubmitTaskToExecutor() { + if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) { + numTasks.incrementAndGet(); + completionService.submit(() -> { + FileStatus fileStatus1; + while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS)) + != null) { + processDirectoryTree(fileStatus1.getPath()); + } + return null; + }); } } - @VisibleForTesting - void registerListener(Listener listener) { - this.listener = listener; - } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java deleted file mode 100644 index 5136fa6757a87..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/Listener.java +++ /dev/null @@ -1,21 +0,0 @@ -package org.apache.hadoop.fs.azurebfs.utils; - -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.atomic.AtomicInteger; - -public interface Listener { - void verifyThreadCount(AtomicInteger numTasks, ThreadPoolExecutor executor, - int maxThreads); - - void checkShutdown(int activeCount); - - void checkAllTasksComplete(AtomicInteger numTasks, int activeCount); - - boolean shouldInterrupt(); - - void setInterrupted(); - - boolean isInterrupted(); - - void checkInterrupt(); -} diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index 7df35fe8ee436..439e2259422f3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -19,19 +19,15 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; +import java.lang.reflect.Field; import java.util.ArrayList; import java.util.List; import java.util.Random; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.*; +import java.util.concurrent.atomic.AtomicReference; import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; -import org.apache.hadoop.fs.azurebfs.utils.ExecutorServiceTestUtils; import org.assertj.core.api.Assertions; import org.junit.Test; @@ -44,7 +40,6 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.when; public class TestGetContentSummary extends AbstractAbfsIntegrationTest { @@ -57,7 +52,7 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { "/testFolder/testFolder3/testFolder7", "/testFolder/testFolder3/testFolder6/leafDir", "/testFolderII/listMaxDir", - "/testFolderII/listMaxDir/zFolder"}; + "/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"); @@ -135,16 +130,50 @@ public void testInvalidPath() throws Exception { "testFolder/IntermediateNonExistentPath"))); } + @Test(timeout = 10000) + public void testTimeTaken() throws Exception { + fs.getContentSummary(new Path("/testFolder")); + } + + @Test + public void testConcurrentCallsOnFilesystem() + throws InterruptedException, ExecutionException { + ExecutorService executorService = new ThreadPoolExecutor(1, + 16, 5, TimeUnit.SECONDS, new SynchronousQueue<>()); + ArrayList> futures = new ArrayList<>(); + for (String directory : directories) { + Future future = executorService.submit( + () -> fs.getContentSummary(new Path(directory))); + futures.add(future); + } + int[][] dirCS = {{8, 16, 8 * testBufferSize}, + {0, 2, 2 * testBufferSize}, {2, 6, 2 * testBufferSize}, + {3, 6, 2 * testBufferSize}, {2, 14, 0}, {0, 2, 0}, + {0, 2, 2 * testBufferSize}, {1, 2, 0}, {0, 2, 0}, + {0, 0, 0}, {1, 12, 0}, {0, 2, 0}}; + executorService.shutdown(); + for(int i=0; i - contentSummaryProcessor.getContentSummary(new Path("/"))); + new ContentSummaryProcessor(getAbfsStore(fs)); + Field executorServiceField = + ContentSummaryProcessor.class.getDeclaredField("executorService"); + executorServiceField.setAccessible(true); + ExecutorService fieldValue = (ExecutorService) executorServiceField.get(contentSummaryProcessor); + contentSummaryProcessor.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs); + Assertions.assertThat(((ThreadPoolExecutor) fieldValue).getLargestPoolSize()) + .describedAs("Core size is 1, so max threads at any time should be >=1") + .isGreaterThanOrEqualTo(1); + Assertions.assertThat(((ThreadPoolExecutor) fieldValue).getPoolSize()) + .describedAs("No threads should remain after executor shutdown") + .isEqualTo(0); } private void checkContentSummary(ContentSummary contentSummary, diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ExecutorServiceTestUtils.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ExecutorServiceTestUtils.java deleted file mode 100644 index 106bc932a0a0e..0000000000000 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/utils/ExecutorServiceTestUtils.java +++ /dev/null @@ -1,66 +0,0 @@ -package org.apache.hadoop.fs.azurebfs.utils; - -import org.assertj.core.api.Assertions; - -import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.atomic.AtomicInteger; - -public class ExecutorServiceTestUtils implements Listener { - boolean interrupt; - boolean hasInterrupted = false; - AtomicInteger numInterrupts = new AtomicInteger(0); - public ExecutorServiceTestUtils(boolean testWithInterrupt) { - interrupt = testWithInterrupt; - } - @Override - public void verifyThreadCount(AtomicInteger numTasks, ThreadPoolExecutor executor, - int maxThreads) { - int activeThreads = executor.getActiveCount(); - int taskQueueSize = executor.getQueue().size(); - Assertions.assertThat(activeThreads).isLessThanOrEqualTo(maxThreads); - Assertions.assertThat(activeThreads).isGreaterThanOrEqualTo(Math.min(numTasks.get() - 2, maxThreads)) - .isLessThanOrEqualTo(numTasks.get() + 2); - if (taskQueueSize > maxThreads) { - Assertions.assertThat(activeThreads) - .describedAs("all threads should be active") - .isEqualTo(maxThreads); - } - } - - @Override - public void checkShutdown(int activeCount) { - Assertions.assertThat(activeCount) - .describedAs("All tasks should be halted").isEqualTo(0); - } - - @Override - public void checkAllTasksComplete(AtomicInteger numTasks, int activeCount) { - Assertions.assertThat(numTasks.get()) - .describedAs("Number of tasks should be 0") - .isEqualTo(0); - Assertions.assertThat(activeCount) - .describedAs("All threads should be halted").isEqualTo(0); - } - - @Override - public boolean shouldInterrupt() { - return interrupt; - } - - @Override - public void setInterrupted() { - hasInterrupted = true; - } - - @Override - public boolean isInterrupted() { - return hasInterrupted; - } - - @Override - public void checkInterrupt() { - Assertions.assertThat(isInterrupted()) - .describedAs("Thread should have been interrupted") - .isEqualTo(false); - } -} From e3eaca78e7f5781e4c1fdad12910b7687076d2a0 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Fri, 8 Jan 2021 12:21:12 +0530 Subject: [PATCH 14/33] clean up --- .../services/ContentSummaryProcessor.java | 7 ------- .../azurebfs/services/TestGetContentSummary.java | 16 +++++----------- 2 files changed, 5 insertions(+), 18 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 706dd4cf03666..6f974634b2c45 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -19,17 +19,10 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Set; import java.util.concurrent.*; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; -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; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index 439e2259422f3..0e8b64cc56a6b 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -24,22 +24,18 @@ import java.util.List; import java.util.Random; import java.util.concurrent.*; -import java.util.concurrent.atomic.AtomicReference; -import org.apache.hadoop.fs.azurebfs.AbstractAbfsIntegrationTest; -import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystem; 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; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; public class TestGetContentSummary extends AbstractAbfsIntegrationTest { @@ -126,8 +122,6 @@ public void testDirOverListMaxResultsItems() public void testInvalidPath() throws Exception { intercept(IOException.class, () -> fs.getContentSummary(new Path( "/nonExistentPath"))); - intercept(IOException.class, () -> fs.getContentSummary(new Path( - "testFolder/IntermediateNonExistentPath"))); } @Test(timeout = 10000) @@ -136,7 +130,7 @@ public void testTimeTaken() throws Exception { } @Test - public void testConcurrentCallsOnFilesystem() + public void testConcurrentGetContentSummaryCalls() throws InterruptedException, ExecutionException { ExecutorService executorService = new ThreadPoolExecutor(1, 16, 5, TimeUnit.SECONDS, new SynchronousQueue<>()); @@ -148,9 +142,9 @@ public void testConcurrentCallsOnFilesystem() } int[][] dirCS = {{8, 16, 8 * testBufferSize}, {0, 2, 2 * testBufferSize}, {2, 6, 2 * testBufferSize}, - {3, 6, 2 * testBufferSize}, {2, 14, 0}, {0, 2, 0}, + {3, 6, 2 * testBufferSize}, {2, numFilesForListMaxTest + 4, 0}, {0, 2, 0}, {0, 2, 2 * testBufferSize}, {1, 2, 0}, {0, 2, 0}, - {0, 0, 0}, {1, 12, 0}, {0, 2, 0}}; + {0, 0, 0}, {1, numFilesForListMaxTest + 2, 0}, {0, 2, 0}}; executorService.shutdown(); for(int i=0; i Date: Fri, 8 Jan 2021 18:17:59 +0530 Subject: [PATCH 15/33] minor changes --- .../fs/azurebfs/AzureBlobFileSystem.java | 4 +- .../services/ContentSummaryProcessor.java | 29 ++++++++++---- .../services/TestGetContentSummary.java | 38 +++++++++++-------- 3 files changed, 46 insertions(+), 25 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index ab7f5acbabdec..5791f77af5fc6 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -377,8 +377,8 @@ public ContentSummary getContentSummary(Path path) throws IOException { .fileCount(contentSummary.getFileCount()) .spaceConsumed(contentSummary.getSpaceConsumed()).build(); } catch (InterruptedException | ExecutionException e) { - LOG.debug((e instanceof InterruptedException || e.getCause() instanceof InterruptedException)? - "Thread interrupted": e.getCause().getMessage()); + LOG.debug((e instanceof InterruptedException || e.getCause() instanceof InterruptedException) + ? "Thread interrupted" : e.getCause().getMessage()); throw new IOException(e); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 6f974634b2c45..92534b7f90e01 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -19,10 +19,20 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; -import java.util.concurrent.*; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.CompletionService; +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; @@ -40,6 +50,7 @@ public class ContentSummaryProcessor { private final CompletionService completionService = new ExecutorCompletionService<>( executorService); private final LinkedBlockingQueue queue = new LinkedBlockingQueue<>(); + private final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class); private static final int POLL_TIMEOUT = 100; public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { @@ -48,19 +59,21 @@ public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { public ABFSContentSummary getContentSummary(Path path) throws IOException, ExecutionException, InterruptedException { - - processDirectoryTree(path); - try { + processDirectoryTree(path); while (!queue.isEmpty() || numTasks.get() > 0) { - completionService.take().get(); - numTasks.decrementAndGet(); + 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 { - numTasks.decrementAndGet(); executorService.shutdown(); + executorService.awaitTermination(1, TimeUnit.SECONDS); } - executorService.awaitTermination(1, TimeUnit.SECONDS); return new ABFSContentSummary(totalBytes.get(), directoryCount.get(), fileCount.get(), totalBytes.get()); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index 0e8b64cc56a6b..1e2e4fa888a07 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -23,7 +23,13 @@ import java.util.ArrayList; import java.util.List; import java.util.Random; -import java.util.concurrent.*; +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; @@ -66,6 +72,7 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { 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(); @@ -124,37 +131,31 @@ public void testInvalidPath() throws Exception { "/nonExistentPath"))); } - @Test(timeout = 10000) - public void testTimeTaken() throws Exception { - fs.getContentSummary(new Path("/testFolder")); - } - @Test public void testConcurrentGetContentSummaryCalls() throws InterruptedException, ExecutionException { ExecutorService executorService = new ThreadPoolExecutor(1, - 16, 5, TimeUnit.SECONDS, new SynchronousQueue<>()); + maxThreads, 5, TimeUnit.SECONDS, new SynchronousQueue<>()); ArrayList> futures = new ArrayList<>(); for (String directory : directories) { Future future = executorService.submit( () -> fs.getContentSummary(new Path(directory))); futures.add(future); } - int[][] dirCS = {{8, 16, 8 * testBufferSize}, - {0, 2, 2 * testBufferSize}, {2, 6, 2 * testBufferSize}, - {3, 6, 2 * testBufferSize}, {2, numFilesForListMaxTest + 4, 0}, {0, 2, 0}, - {0, 2, 2 * testBufferSize}, {1, 2, 0}, {0, 2, 0}, - {0, 0, 0}, {1, numFilesForListMaxTest + 2, 0}, {0, 2, 0}}; + 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 contentSummaryProcessor1.getContentSummary(new Path("/invalidPath"))); + Assertions.assertThat(((ThreadPoolExecutor) fieldValue).getLargestPoolSize()) + .describedAs("No task was submitted") + .isEqualTo(0); } private void checkContentSummary(ContentSummary contentSummary, From 48d0607da7fe088c8291815a04cb746461abb5a6 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Sat, 9 Jan 2021 10:31:46 +0530 Subject: [PATCH 16/33] rm thread test --- .../services/TestGetContentSummary.java | 25 ------------------- 1 file changed, 25 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index 1e2e4fa888a07..0c3fc16ef02fe 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -19,7 +19,6 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; -import java.lang.reflect.Field; import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -154,30 +153,6 @@ public void testConcurrentGetContentSummaryCalls() } } - @Test - public void testExecutorShutdown() throws Exception { - ContentSummaryProcessor contentSummaryProcessor = - new ContentSummaryProcessor(getAbfsStore(fs)); - Field executorServiceField = - ContentSummaryProcessor.class.getDeclaredField("executorService"); - executorServiceField.setAccessible(true); - ExecutorService fieldValue = (ExecutorService) executorServiceField.get(contentSummaryProcessor); - contentSummaryProcessor.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs); - Assertions.assertThat(((ThreadPoolExecutor) fieldValue).getLargestPoolSize()) - .describedAs("Core size is 1, so max threads at any time should be >=1") - .isGreaterThanOrEqualTo(1); - Assertions.assertThat(((ThreadPoolExecutor) fieldValue).getPoolSize()) - .describedAs("No threads should remain after executor shutdown") - .isEqualTo(0); - ContentSummaryProcessor contentSummaryProcessor1 = - new ContentSummaryProcessor(getAbfsStore(fs)); - fieldValue = (ExecutorService) executorServiceField.get(contentSummaryProcessor1); - intercept(IOException.class, () -> contentSummaryProcessor1.getContentSummary(new Path("/invalidPath"))); - Assertions.assertThat(((ThreadPoolExecutor) fieldValue).getLargestPoolSize()) - .describedAs("No task was submitted") - .isEqualTo(0); - } - private void checkContentSummary(ContentSummary contentSummary, long directoryCount, long fileCount, long byteCount) { Assertions.assertThat(contentSummary.getDirectoryCount()) From a10be0011da1fed8ad176eaf2f72f7ef381798e4 Mon Sep 17 00:00:00 2001 From: Sumangala Date: Sun, 10 Jan 2021 21:13:41 +0530 Subject: [PATCH 17/33] checkstyle --- .../fs/azurebfs/services/ContentSummaryProcessor.java | 11 +++++------ .../fs/azurebfs/services/TestGetContentSummary.java | 2 +- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 92534b7f90e01..5066d64f1ece1 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -19,10 +19,10 @@ package org.apache.hadoop.fs.azurebfs.services; import java.io.IOException; -import java.util.concurrent.ExecutorService; +import java.util.concurrent.CompletionService; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; -import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutorService; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; @@ -50,7 +50,7 @@ public class ContentSummaryProcessor { private final CompletionService completionService = new ExecutorCompletionService<>( executorService); private final LinkedBlockingQueue queue = new LinkedBlockingQueue<>(); - private final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class); + private static final Logger LOG = LoggerFactory.getLogger(ContentSummaryProcessor.class); private static final int POLL_TIMEOUT = 100; public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { @@ -63,7 +63,7 @@ public ABFSContentSummary getContentSummary(Path path) 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()); + queue.size(), numTasks, ((ThreadPoolExecutor) executorService).getActiveCount()); try { completionService.take().get(); } finally { @@ -71,8 +71,7 @@ public ABFSContentSummary getContentSummary(Path path) } } } finally { - executorService.shutdown(); - executorService.awaitTermination(1, TimeUnit.SECONDS); + executorService.shutdownNow(); } return new ABFSContentSummary(totalBytes.get(), directoryCount.get(), diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index 0c3fc16ef02fe..a5ca29900d8cc 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -55,7 +55,7 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { "/testFolderII/listMaxDir", "/testFolderII/listMaxDir/" + DEFAULT_AZURE_LIST_MAX_RESULTS/2 + "_mid_folder"}; - private final Path pathToFile = new Path("/testFolder/test1");; + 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"); From bc276b29746dcbfec77b659de8f9d4e43d83467f Mon Sep 17 00:00:00 2001 From: Sumangala Date: Tue, 12 Jan 2021 11:04:32 +0530 Subject: [PATCH 18/33] revw changes + doc --- .../fs/azurebfs/AzureBlobFileSystem.java | 15 +++- .../services/ContentSummaryProcessor.java | 46 +++++++--- .../services/TestGetContentSummary.java | 83 +++++++++++-------- 3 files changed, 94 insertions(+), 50 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 5791f77af5fc6..b3b496bab4964 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -367,6 +367,13 @@ public boolean delete(final Path f, final boolean recursive) throws IOException } + /** + * Performs a series of listStatus operations to the count of directories, + * files and total number of bytes under a given path + * @param path: The given path + * @return ContentSummary + * @throws IOException + */ @Override public ContentSummary getContentSummary(Path path) throws IOException { try { @@ -376,10 +383,12 @@ public ContentSummary getContentSummary(Path path) throws IOException { .directoryCount(contentSummary.getDirectoryCount()) .fileCount(contentSummary.getFileCount()) .spaceConsumed(contentSummary.getSpaceConsumed()).build(); - } catch (InterruptedException | ExecutionException e) { - LOG.debug((e instanceof InterruptedException || e.getCause() instanceof InterruptedException) - ? "Thread interrupted" : e.getCause().getMessage()); + } catch (InterruptedException e) { + LOG.debug("Thread interrupted"); throw new IOException(e); + } catch(ExecutionException ex) { + LOG.debug(ex.getCause().getMessage()); + throw new IOException(ex); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 5066d64f1ece1..ddd68b207d868 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -38,20 +38,27 @@ import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary; +/** + * Processes a given path for count of subdirectories, files and total number + * of bytes. + */ 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 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 completionService = new ExecutorCompletionService<>( - executorService); + private final ExecutorService executorService = new ThreadPoolExecutor( + CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS, + new SynchronousQueue<>()); + private final CompletionService completionService = + new ExecutorCompletionService<>(executorService); private final LinkedBlockingQueue 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; @@ -62,22 +69,31 @@ public ABFSContentSummary getContentSummary(Path path) 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(); + 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); @@ -97,13 +113,21 @@ private void processDirectory() { directoryCount.incrementAndGet(); } + /** + * Increments file count and byte count + * @param fileStatus: Provides file size to update byte count + */ private void processFile(FileStatus fileStatus) { fileCount.incrementAndGet(); totalBytes.addAndGet(fileStatus.getLen()); } + /** + * Submit task for processing a subdirectory based on current size of + * filestatus queue and number of already submitted tasks + */ private synchronized void conditionalSubmitTaskToExecutor() { - if (!queue.isEmpty() && numTasks.get() < NUM_THREADS) { + if (!queue.isEmpty() && numTasks.get() < MAX_THREAD_COUNT) { numTasks.incrementAndGet(); completionService.submit(() -> { FileStatus fileStatus1; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index a5ca29900d8cc..a98db1d3f02c2 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -44,9 +44,17 @@ 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 final String[] directories = {"/testFolder", "/testFolder/testFolder1", - "/testFolder/testFolder2", "/testFolder/testFolder3", "/testFolderII", + "/testFolder/testFolder2", + "/testFolder/testFolder3", + "/testFolderII", "/testFolder/testFolder2/testFolder4", "/testFolder/testFolder2/testFolder5", "/testFolder/testFolder3/testFolder6", @@ -54,7 +62,6 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { "/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 = @@ -67,11 +74,7 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { "/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; + private final byte[] b = new byte[TEST_BUFFER_SIZE]; public TestGetContentSummary() throws Exception { createDirectoryStructure(); @@ -82,46 +85,46 @@ public TestGetContentSummary() throws Exception { public void testFilesystemRoot() throws IOException { int fileCount = - (directories.length - 2) * filesPerDirectory + numFilesForListMaxTest; + (directories.length - 2) * FILES_PER_DIRECTORY + NUM_FILES_FOR_LIST_MAX_TEST; ContentSummary contentSummary = fs.getContentSummary(new Path("/")); - checkContentSummary(contentSummary, directories.length, fileCount, - dirsWithNonEmptyFiles.length * filesPerDirectory * testBufferSize); + verifyContentSummary(contentSummary, directories.length, fileCount, + dirsWithNonEmptyFiles.length * FILES_PER_DIRECTORY * TEST_BUFFER_SIZE); } @Test public void testFileContentSummary() throws IOException { ContentSummary contentSummary = fs.getContentSummary(pathToFile); - checkContentSummary(contentSummary, 0, 1, testBufferSize); + verifyContentSummary(contentSummary, 0, 1, TEST_BUFFER_SIZE); } @Test public void testLeafDir() throws IOException { ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir); - checkContentSummary(contentSummary, 0, 0, 0); + verifyContentSummary(contentSummary, 0, 0, 0); } @Test public void testIntermediateDirWithFilesOnly() throws IOException { ContentSummary contentSummary = fs.getContentSummary(pathToIntermediateDirWithFilesOnly); - checkContentSummary(contentSummary, 0, filesPerDirectory, - testBufferSize * filesPerDirectory); + verifyContentSummary(contentSummary, 0, FILES_PER_DIRECTORY, + TEST_BUFFER_SIZE * FILES_PER_DIRECTORY); } @Test public void testIntermediateDirWithFilesAndSubdirs() throws IOException { ContentSummary contentSummary = fs.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs); - checkContentSummary(contentSummary, 3, 3 * filesPerDirectory, - testBufferSize * filesPerDirectory); + verifyContentSummary(contentSummary, 3, 3 * FILES_PER_DIRECTORY, + TEST_BUFFER_SIZE * FILES_PER_DIRECTORY); } @Test public void testDirOverListMaxResultsItems() throws IOException { - checkContentSummary( + verifyContentSummary( fs.getContentSummary(pathToListMaxDir), 1, - numFilesForListMaxTest + filesPerDirectory, 0); + NUM_FILES_FOR_LIST_MAX_TEST + FILES_PER_DIRECTORY, 0); } @Test @@ -133,36 +136,44 @@ public void testInvalidPath() throws Exception { @Test public void testConcurrentGetContentSummaryCalls() throws InterruptedException, ExecutionException { - ExecutorService executorService = new ThreadPoolExecutor(1, - maxThreads, 5, TimeUnit.SECONDS, new SynchronousQueue<>()); + ExecutorService executorService = new ThreadPoolExecutor(1, MAX_THREADS, + 5, TimeUnit.SECONDS, new SynchronousQueue<>()); ArrayList> futures = new ArrayList<>(); for (String directory : directories) { Future 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(); + int[][] dirCS = {{8, 8 * FILES_PER_DIRECTORY, 8 * TEST_BUFFER_SIZE}, + {0, FILES_PER_DIRECTORY, 2 * TEST_BUFFER_SIZE}, + {2, 3 * FILES_PER_DIRECTORY, 2 * TEST_BUFFER_SIZE}, + {3, 3 * FILES_PER_DIRECTORY, 2 * TEST_BUFFER_SIZE}, + {2, NUM_FILES_FOR_LIST_MAX_TEST + 2 * FILES_PER_DIRECTORY, 0}, + {0, FILES_PER_DIRECTORY, 0}, + {0, FILES_PER_DIRECTORY, FILES_PER_DIRECTORY * TEST_BUFFER_SIZE}, + {1, FILES_PER_DIRECTORY, 0}, + {0, FILES_PER_DIRECTORY, 0}, + {0, 0, 0}, + {1, NUM_FILES_FOR_LIST_MAX_TEST + 2, 0}, + {0, FILES_PER_DIRECTORY, 0}}; + for (int i=0; i Date: Tue, 12 Jan 2021 15:51:59 +0530 Subject: [PATCH 19/33] javadoc --- .../apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java | 7 ++++--- .../fs/azurebfs/services/ContentSummaryProcessor.java | 10 ++++++---- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index b3b496bab4964..e08ce3ce7e47a 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -368,11 +368,12 @@ public boolean delete(final Path f, final boolean recursive) throws IOException } /** - * Performs a series of listStatus operations to the count of directories, + * Returns a ContentSummary instance containing the count of directories, * files and total number of bytes under a given path - * @param path: The given path + * @param path The given path * @return ContentSummary - * @throws IOException + * @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 { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index ddd68b207d868..34b7c509d2dcc 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -38,10 +38,6 @@ import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary; -/** - * Processes a given path for count of subdirectories, files and total number - * of bytes. - */ public class ContentSummaryProcessor { private static final int CORE_POOL_SIZE = 1; private static final int MAX_THREAD_COUNT = 16; @@ -60,6 +56,12 @@ public class ContentSummaryProcessor { new ExecutorCompletionService<>(executorService); private final LinkedBlockingQueue 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(AzureBlobFileSystemStore abfsStore) { this.abfsStore = abfsStore; } From 9070413184bf5c9b2ed472fd805646afe002f3e9 Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 12 Jan 2021 20:43:41 +0530 Subject: [PATCH 20/33] trigger yetus From 041d9bcd4ef0cc8e51973864c545cbaaa0bec92c Mon Sep 17 00:00:00 2001 From: sumangala Date: Sun, 7 Feb 2021 20:10:53 +0530 Subject: [PATCH 21/33] use listingsupport to abstract store --- .../hadoop/fs/azurebfs/services/ContentSummaryProcessor.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 34b7c509d2dcc..6c29118a163b4 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -48,7 +48,7 @@ public class ContentSummaryProcessor { 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 ListingSupport abfsStore; private final ExecutorService executorService = new ThreadPoolExecutor( CORE_POOL_SIZE, MAX_THREAD_COUNT, KEEP_ALIVE_TIME, TimeUnit.SECONDS, new SynchronousQueue<>()); @@ -62,7 +62,7 @@ public class ContentSummaryProcessor { * @param abfsStore Instance of AzureBlobFileSystemStore, used to make * listStatus calls to server */ - public ContentSummaryProcessor(AzureBlobFileSystemStore abfsStore) { + public ContentSummaryProcessor(ListingSupport abfsStore) { this.abfsStore = abfsStore; } From 4be7b19a33406e63b4b7f694eae5d619c1751f96 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 22 Feb 2021 17:08:39 +0530 Subject: [PATCH 22/33] checkstyle --- .../hadoop/fs/azurebfs/services/ContentSummaryProcessor.java | 1 - 1 file changed, 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 6c29118a163b4..302a0e5a99e97 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -35,7 +35,6 @@ 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 { From d21b58a9a8e2730bb137f21ce257e30e5a5b49f8 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 22 Feb 2021 22:04:36 +0530 Subject: [PATCH 23/33] import order --- .../hadoop/fs/azurebfs/services/TestGetContentSummary.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index a98db1d3f02c2..68ab8a069f6e7 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -22,8 +22,8 @@ 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.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.SynchronousQueue; From 237843103950ebb34d0161a6d005b893a941ae10 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 19 Apr 2021 14:36:02 +0530 Subject: [PATCH 24/33] log ex --- .../java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index f9571f3e651db..4c0f90ba7b24e 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -418,7 +418,7 @@ public ContentSummary getContentSummary(Path path) throws IOException { LOG.debug("Thread interrupted"); throw new IOException(e); } catch(ExecutionException ex) { - LOG.debug(ex.getCause().getMessage()); + LOG.debug("GetContentSummary failed with error " + ex.getCause().getMessage()); throw new IOException(ex); } } From fa34b5788479095b23df3a7f9a34f6592c45f6ed Mon Sep 17 00:00:00 2001 From: sumangala Date: Tue, 11 May 2021 15:15:16 +0530 Subject: [PATCH 25/33] rm abfs cs --- .../fs/azurebfs/AzureBlobFileSystem.java | 11 +---- .../services/ContentSummaryProcessor.java | 10 ++-- .../fs/azurebfs/utils/ABFSContentSummary.java | 49 ------------------- 3 files changed, 8 insertions(+), 62 deletions(-) delete mode 100644 hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ABFSContentSummary.java diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index a072b20ddb104..52b2cd5c0fdcb 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -56,7 +56,6 @@ import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.CommonPathCapabilities; import org.apache.hadoop.fs.ContentSummary; -import org.apache.hadoop.fs.ContentSummary.Builder; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -79,7 +78,6 @@ import org.apache.hadoop.fs.azurebfs.security.AbfsDelegationTokenManager; import org.apache.hadoop.fs.azurebfs.services.AbfsCounters; import org.apache.hadoop.fs.azurebfs.services.ContentSummaryProcessor; -import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary; import org.apache.hadoop.fs.impl.AbstractFSBuilderImpl; import org.apache.hadoop.fs.impl.OpenFileParameters; import org.apache.hadoop.fs.permission.AclEntry; @@ -410,17 +408,12 @@ 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(); + return (new ContentSummaryProcessor(abfsStore)).getContentSummary(path); } catch (InterruptedException e) { LOG.debug("Thread interrupted"); throw new IOException(e); } catch(ExecutionException ex) { - LOG.debug("GetContentSummary failed with error " + ex.getCause().getMessage()); + LOG.debug("GetContentSummary failed with error: {}", ex.getMessage()); throw new IOException(ex); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 302a0e5a99e97..7b413ba983b84 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -33,9 +33,9 @@ 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; -import org.apache.hadoop.fs.azurebfs.utils.ABFSContentSummary; public class ContentSummaryProcessor { private static final int CORE_POOL_SIZE = 1; @@ -65,7 +65,7 @@ public ContentSummaryProcessor(ListingSupport abfsStore) { this.abfsStore = abfsStore; } - public ABFSContentSummary getContentSummary(Path path) + public ContentSummary getContentSummary(Path path) throws IOException, ExecutionException, InterruptedException { try { processDirectoryTree(path); @@ -83,8 +83,10 @@ public ABFSContentSummary getContentSummary(Path path) 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()); + ContentSummary.Builder builder = new ContentSummary.Builder() + .directoryCount(directoryCount.get()).fileCount(fileCount.get()) + .length(totalBytes.get()).spaceConsumed(totalBytes.get()); + return builder.build(); } /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ABFSContentSummary.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ABFSContentSummary.java deleted file mode 100644 index 4ac505569291e..0000000000000 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/utils/ABFSContentSummary.java +++ /dev/null @@ -1,49 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.azurebfs.utils; - -public class ABFSContentSummary { - private final long length; - private final long directoryCount; - private final long fileCount; - private final long spaceConsumed; - - public ABFSContentSummary(long length, long directoryCount, long fileCount, - long spaceConsumed) { - this.length = length; - this.directoryCount = directoryCount; - this.fileCount = fileCount; - this.spaceConsumed = spaceConsumed; - } - - public long getLength() { - return length; - } - - public long getDirectoryCount() { - return directoryCount; - } - - public long getFileCount() { - return fileCount; - } - public long getSpaceConsumed() { - return spaceConsumed; - } -} From aa48086d7d7abd4fef4a037f684fe99cb68797fc Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 12 May 2021 15:53:14 +0530 Subject: [PATCH 26/33] test fix --- .../services/ContentSummaryProcessor.java | 43 ++++++ .../services/TestGetContentSummary.java | 139 +++++++++--------- 2 files changed, 114 insertions(+), 68 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 7b413ba983b84..1375bcbd42a12 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -23,7 +23,9 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; +import java.util.concurrent.ForkJoinPool; import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RecursiveAction; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -67,6 +69,7 @@ public ContentSummaryProcessor(ListingSupport abfsStore) { public ContentSummary getContentSummary(Path path) throws IOException, ExecutionException, InterruptedException { +// return gcs_fjp(path); try { processDirectoryTree(path); while (!queue.isEmpty() || numTasks.get() > 0) { @@ -89,6 +92,46 @@ public ContentSummary getContentSummary(Path path) return builder.build(); } + class GCS extends RecursiveAction { + Path path; + GCS (Path path) { + this.path = path; + } + @Override + protected void compute() { + try { + for (FileStatus fileStatus1 : abfsStore.listStatus(path)) { + if (fileStatus1.isDirectory()) { + processDirectory(); + new GCS(fileStatus1.getPath()).fork(); + } else { + processFile(fileStatus1); + } + } + } catch (IOException e) { + e.printStackTrace(); + } + } + } + + public ContentSummary gcs_fjp (Path path) + throws InterruptedException, IOException { + ForkJoinPool forkJoinPool = new ForkJoinPool(); + for (FileStatus fileStatus : abfsStore.listStatus(path)) { + if (fileStatus.isDirectory()) { + processDirectory(); + forkJoinPool.invoke(new GCS(fileStatus.getPath())); + } else { + processFile(fileStatus); + } + } + forkJoinPool.awaitTermination(1, TimeUnit.SECONDS); + 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 diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index 68ab8a069f6e7..b5887805b687f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -47,119 +47,128 @@ 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_FILES_FOR_LIST_MAX_TEST = 10; +// 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", - "/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(); + "/testFolder/testFolder3/testFolder7"}; + private final byte[] b = new byte[TEST_BUFFER_SIZE]; public TestGetContentSummary() throws Exception { - createDirectoryStructure(); new Random().nextBytes(b); } @Test public void testFilesystemRoot() - throws IOException { - int fileCount = - (directories.length - 2) * FILES_PER_DIRECTORY + NUM_FILES_FOR_LIST_MAX_TEST; + 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, - dirsWithNonEmptyFiles.length * FILES_PER_DIRECTORY * TEST_BUFFER_SIZE); + directories.length * TEST_BUFFER_SIZE); } @Test public void testFileContentSummary() throws IOException { - ContentSummary contentSummary = fs.getContentSummary(pathToFile); + 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 { - ContentSummary contentSummary = fs.getContentSummary(pathToLeafDir); + 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 { + 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(pathToIntermediateDirWithFilesOnly); + fs.getContentSummary(intermediateDir); verifyContentSummary(contentSummary, 0, FILES_PER_DIRECTORY, - TEST_BUFFER_SIZE * FILES_PER_DIRECTORY); + TEST_BUFFER_SIZE); } @Test - public void testIntermediateDirWithFilesAndSubdirs() throws IOException { + 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")); ContentSummary contentSummary = - fs.getContentSummary(pathToIntermediateDirWithFilesAndSubdirs); - verifyContentSummary(contentSummary, 3, 3 * FILES_PER_DIRECTORY, - TEST_BUFFER_SIZE * FILES_PER_DIRECTORY); + fs.getContentSummary(intermediateDir); + verifyContentSummary(contentSummary, 1, FILES_PER_DIRECTORY, + TEST_BUFFER_SIZE); } @Test public void testDirOverListMaxResultsItems() - throws IOException { + 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 + FILES_PER_DIRECTORY, 0); + 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 { - ExecutorService executorService = new ThreadPoolExecutor(1, MAX_THREADS, - 5, TimeUnit.SECONDS, new SynchronousQueue<>()); + throws InterruptedException, ExecutionException, IOException { + ExecutorService executorService = new ThreadPoolExecutor(1, MAX_THREADS, 5, + TimeUnit.SECONDS, new SynchronousQueue<>()); ArrayList> futures = new ArrayList<>(); - for (String directory : directories) { + createDirectoryStructure(); + for (int i = 0; i < NUM_CONCURRENT_CALLS; i++) { Future future = executorService.submit( - () -> fs.getContentSummary(new Path(directory))); + () -> getFileSystem().getContentSummary(new Path("/testFolder"))); futures.add(future); } - int[][] dirCS = {{8, 8 * FILES_PER_DIRECTORY, 8 * TEST_BUFFER_SIZE}, - {0, FILES_PER_DIRECTORY, 2 * TEST_BUFFER_SIZE}, - {2, 3 * FILES_PER_DIRECTORY, 2 * TEST_BUFFER_SIZE}, - {3, 3 * FILES_PER_DIRECTORY, 2 * TEST_BUFFER_SIZE}, - {2, NUM_FILES_FOR_LIST_MAX_TEST + 2 * FILES_PER_DIRECTORY, 0}, - {0, FILES_PER_DIRECTORY, 0}, - {0, FILES_PER_DIRECTORY, FILES_PER_DIRECTORY * TEST_BUFFER_SIZE}, - {1, FILES_PER_DIRECTORY, 0}, - {0, FILES_PER_DIRECTORY, 0}, - {0, 0, 0}, - {1, NUM_FILES_FOR_LIST_MAX_TEST + 2, 0}, - {0, FILES_PER_DIRECTORY, 0}}; - - for (int i=0; i> tasks = new ArrayList<>(); ExecutorService es = Executors.newFixedThreadPool(10); for (int i = 0; i < numFiles; i++) { @@ -210,6 +209,10 @@ private void populateDirWithFiles(Path directory, int numFiles) for (Future task : tasks) { task.get(); } + FSDataOutputStream out = getFileSystem() + .append(new Path(directory + "/test0")); + out.write(b); + out.close(); es.shutdownNow(); } } From f320785b00896eb8a59bc4ce764f09788a23b307 Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 12 May 2021 16:04:35 +0530 Subject: [PATCH 27/33] clean up --- .../services/ContentSummaryProcessor.java | 43 ------------------- .../services/TestGetContentSummary.java | 17 +++++--- 2 files changed, 10 insertions(+), 50 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 1375bcbd42a12..7b413ba983b84 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -23,9 +23,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.ExecutorService; -import java.util.concurrent.ForkJoinPool; import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.RecursiveAction; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -69,7 +67,6 @@ public ContentSummaryProcessor(ListingSupport abfsStore) { public ContentSummary getContentSummary(Path path) throws IOException, ExecutionException, InterruptedException { -// return gcs_fjp(path); try { processDirectoryTree(path); while (!queue.isEmpty() || numTasks.get() > 0) { @@ -92,46 +89,6 @@ public ContentSummary getContentSummary(Path path) return builder.build(); } - class GCS extends RecursiveAction { - Path path; - GCS (Path path) { - this.path = path; - } - @Override - protected void compute() { - try { - for (FileStatus fileStatus1 : abfsStore.listStatus(path)) { - if (fileStatus1.isDirectory()) { - processDirectory(); - new GCS(fileStatus1.getPath()).fork(); - } else { - processFile(fileStatus1); - } - } - } catch (IOException e) { - e.printStackTrace(); - } - } - } - - public ContentSummary gcs_fjp (Path path) - throws InterruptedException, IOException { - ForkJoinPool forkJoinPool = new ForkJoinPool(); - for (FileStatus fileStatus : abfsStore.listStatus(path)) { - if (fileStatus.isDirectory()) { - processDirectory(); - forkJoinPool.invoke(new GCS(fileStatus.getPath())); - } else { - processFile(fileStatus); - } - } - forkJoinPool.awaitTermination(1, TimeUnit.SECONDS); - 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 diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java index b5887805b687f..1c0e89007a152 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java @@ -22,7 +22,9 @@ 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; @@ -47,8 +49,8 @@ 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 = 10; -// DEFAULT_AZURE_LIST_MAX_RESULTS + 10; + 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", @@ -156,17 +158,18 @@ public void testInvalidPath() throws Exception { @Test public void testConcurrentGetContentSummaryCalls() throws InterruptedException, ExecutionException, IOException { + AzureBlobFileSystem fs = getFileSystem(); ExecutorService executorService = new ThreadPoolExecutor(1, MAX_THREADS, 5, TimeUnit.SECONDS, new SynchronousQueue<>()); - ArrayList> futures = new ArrayList<>(); + CompletionService completionService = + new ExecutorCompletionService<>(executorService); createDirectoryStructure(); for (int i = 0; i < NUM_CONCURRENT_CALLS; i++) { - Future future = executorService.submit( - () -> getFileSystem().getContentSummary(new Path("/testFolder"))); - futures.add(future); + completionService.submit(() -> fs.getContentSummary(new Path( + "/testFolder"))); } for (int i = 0; i < NUM_CONCURRENT_CALLS; i++) { - ContentSummary contentSummary = futures.get(i).get(); + ContentSummary contentSummary = completionService.take().get(); verifyContentSummary(contentSummary, 7, 8 * FILES_PER_DIRECTORY, 8 * TEST_BUFFER_SIZE); } From a718cbd669cc6c86b75ed802d9d8f39ccd71d138 Mon Sep 17 00:00:00 2001 From: sumangala Date: Wed, 25 Aug 2021 09:37:48 +0530 Subject: [PATCH 28/33] address revw comments --- .../fs/azurebfs/AzureBlobFileSystem.java | 129 +++++++++--------- .../services/ContentSummaryProcessor.java | 15 +- ...mmary.java => ITestGetContentSummary.java} | 61 ++++----- 3 files changed, 99 insertions(+), 106 deletions(-) rename hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/{TestGetContentSummary.java => ITestGetContentSummary.java} (84%) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 5d7cee38d5ece..5008476f9c058 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -22,6 +22,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.io.InterruptedIOException; import java.io.OutputStream; import java.net.HttpURLConnection; import java.net.URI; @@ -126,6 +127,9 @@ public class AzureBlobFileSystem extends FileSystem private String clientCorrelationId; private TracingHeaderFormat tracingHeaderFormat; private Listener listener; + private final ExecutorService contentSummaryExecutorService = Executors.newCachedThreadPool(); +// private final ExecutorService contentSummaryExecutorService = new ThreadPoolExecutor( +// 0, 16, 5, TimeUnit.SECONDS, new SynchronousQueue<>()); @Override public void initialize(URI uri, Configuration configuration) @@ -142,8 +146,7 @@ public void initialize(URI uri, Configuration configuration) configuration, abfsCounters); LOG.trace("AzureBlobFileSystemStore init complete"); - final AbfsConfiguration abfsConfiguration = abfsStore - .getAbfsConfiguration(); + final AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); clientCorrelationId = TracingContext.validateClientCorrelationID( abfsConfiguration.getClientCorrelationId()); tracingHeaderFormat = abfsConfiguration.getTracingHeaderFormat(); @@ -151,8 +154,10 @@ public void initialize(URI uri, Configuration configuration) if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener); - if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), tracingContext) == null) { + fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, + listener); + if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), + tracingContext) == null) { try { this.createFileSystem(tracingContext); } catch (AzureBlobFileSystemException ex) { @@ -180,8 +185,7 @@ public void initialize(URI uri, Configuration configuration) @Override public String toString() { - final StringBuilder sb = new StringBuilder( - "AzureBlobFileSystem{"); + final StringBuilder sb = new StringBuilder("AzureBlobFileSystem{"); sb.append("uri=").append(uri); sb.append(", user='").append(abfsStore.getUser()).append('\''); sb.append(", primaryUserGroup='").append(abfsStore.getPrimaryGroup()).append('\''); @@ -204,7 +208,8 @@ public void registerListener(Listener listener1) { @Override public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { - LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); + LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, + bufferSize); // bufferSize is unused. return open(path, Optional.empty()); } @@ -217,10 +222,10 @@ private FSDataInputStream open(final Path path, try { TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.OPEN, tracingHeaderFormat, listener); - InputStream inputStream = abfsStore - .openFileForRead(qualifiedPath, parameters, statistics, tracingContext); + InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, + parameters, statistics, tracingContext); return new FSDataInputStream(inputStream); - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { checkException(path, ex); return null; } @@ -231,7 +236,8 @@ private FSDataInputStream open(final Path path, * {@link org.apache.hadoop.fs.impl.OpenFileParameters}. Ensure that * FileStatus entered is up-to-date, as it will be used to create the * InputStream (with info such as contentLength, eTag) - * @param path The location of file to be opened + * + * @param path The location of file to be opened * @param parameters OpenFileParameters instance; can hold FileStatus, * Configuration, bufferSize and mandatoryKeys */ @@ -240,22 +246,17 @@ protected CompletableFuture openFileWithOptions( final Path path, final OpenFileParameters parameters) throws IOException { LOG.debug("AzureBlobFileSystem.openFileWithOptions path: {}", path); AbstractFSBuilderImpl.rejectUnknownMandatoryKeys( - parameters.getMandatoryKeys(), - Collections.emptySet(), - "for " + path); - return LambdaUtils.eval( - new CompletableFuture<>(), () -> - open(path, Optional.of(parameters))); + parameters.getMandatoryKeys(), Collections.emptySet(), "for " + path); + return LambdaUtils.eval(new CompletableFuture<>(), () -> open(path, Optional.of(parameters))); } @Override - public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize, - final short replication, final long blockSize, final Progressable progress) throws IOException { - LOG.debug("AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", - f, - permission, - overwrite, - blockSize); + public FSDataOutputStream create(final Path f, final FsPermission permission, + final boolean overwrite, final int bufferSize, final short replication, + final long blockSize, final Progressable progress) throws IOException { + LOG.debug( + "AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", + f, permission, overwrite, blockSize); statIncrement(CALL_CREATE); trailingPeriodCheck(f); @@ -264,13 +265,14 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi try { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.CREATE, overwrite, tracingHeaderFormat, listener); + fileSystemId, FSOperationType.CREATE, overwrite, tracingHeaderFormat, + listener); OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); statIncrement(FILES_CREATED); return new FSDataOutputStream(outputStream, statistics); - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { checkException(f, ex); return null; } @@ -290,11 +292,11 @@ public FSDataOutputStream createNonRecursive(final Path f, final FsPermission pe final FileStatus parentFileStatus = tryGetFileStatus(parent, tracingContext); if (parentFileStatus == null) { - throw new FileNotFoundException("Cannot create file " - + f.getName() + " because parent folder does not exist."); + throw new FileNotFoundException("Cannot create file " + f.getName() + " because parent folder does not exist."); } - return create(f, permission, overwrite, bufferSize, replication, blockSize, progress); + return create(f, permission, overwrite, bufferSize, replication, blockSize, + progress); } @Override @@ -305,41 +307,38 @@ public FSDataOutputStream createNonRecursive(final Path f, final FsPermission pe // Check if file should be appended or overwritten. Assume that the file // is overwritten on if the CREATE and OVERWRITE create flags are set. - final EnumSet createflags = - EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE); + final EnumSet createflags = EnumSet.of(CreateFlag.CREATE, + CreateFlag.OVERWRITE); final boolean overwrite = flags.containsAll(createflags); // Delegate the create non-recursive call. - return this.createNonRecursive(f, permission, overwrite, - bufferSize, replication, blockSize, progress); + return this.createNonRecursive(f, permission, overwrite, bufferSize, + replication, blockSize, progress); } @Override @SuppressWarnings("deprecation") public FSDataOutputStream createNonRecursive(final Path f, - final boolean overwrite, final int bufferSize, final short replication, final long blockSize, - final Progressable progress) throws IOException { - return this.createNonRecursive(f, FsPermission.getFileDefault(), - overwrite, bufferSize, replication, blockSize, progress); + final boolean overwrite, final int bufferSize, final short replication, + final long blockSize, final Progressable progress) throws IOException { + return this.createNonRecursive(f, FsPermission.getFileDefault(), overwrite, + bufferSize, replication, blockSize, progress); } @Override public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { - LOG.debug( - "AzureBlobFileSystem.append path: {} bufferSize: {}", - f.toString(), - bufferSize); + LOG.debug("AzureBlobFileSystem.append path: {} bufferSize: {}", + f.toString(), bufferSize); statIncrement(CALL_APPEND); Path qualifiedPath = makeQualified(f); try { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.APPEND, tracingHeaderFormat, - listener); - OutputStream outputStream = abfsStore - .openFileForWrite(qualifiedPath, statistics, false, tracingContext); + fileSystemId, FSOperationType.APPEND, tracingHeaderFormat, listener); + OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, + statistics, false, tracingContext); return new FSDataOutputStream(outputStream, statistics); - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { checkException(f, ex); return null; } @@ -362,7 +361,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { fileSystemId, FSOperationType.RENAME, true, tracingHeaderFormat, listener); // rename under same folder; - if(makeQualified(parentFolder).equals(qualifiedDstPath)) { + if (makeQualified(parentFolder).equals(qualifiedDstPath)) { return tryGetFileStatus(qualifiedSrcPath, tracingContext) != null; } @@ -399,17 +398,13 @@ public boolean rename(final Path src, final Path dst) throws IOException { abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, tracingContext); return true; - } catch(AzureBlobFileSystemException ex) { + } catch (AzureBlobFileSystemException ex) { LOG.debug("Rename operation failed. ", ex); - checkException( - src, - ex, - AzureServiceErrorCode.PATH_ALREADY_EXISTS, - AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, - AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, - AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, - AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND, - AzureServiceErrorCode.INTERNAL_OPERATION_ABORT); + checkException(src, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS, + AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, + AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, + AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND, + AzureServiceErrorCode.INTERNAL_OPERATION_ABORT); return false; } @@ -417,8 +412,8 @@ public boolean rename(final Path src, final Path dst) throws IOException { @Override public boolean delete(final Path f, final boolean recursive) throws IOException { - LOG.debug( - "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); + LOG.debug("AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), + recursive); statIncrement(CALL_DELETE); Path qualifiedPath = makeQualified(f); @@ -432,8 +427,7 @@ public boolean delete(final Path f, final boolean recursive) throws IOException try { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.DELETE, tracingHeaderFormat, - listener); + fileSystemId, FSOperationType.DELETE, tracingHeaderFormat, listener); abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { @@ -446,10 +440,12 @@ 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 + * or if there is any issue with the thread pool used + * while processing */ @Override public ContentSummary getContentSummary(Path path) throws IOException { @@ -457,14 +453,15 @@ public ContentSummary getContentSummary(Path path) throws IOException { TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.GET_CONTENT_SUMMARY, true, tracingHeaderFormat, listener); - return (new ContentSummaryProcessor(abfsStore)).getContentSummary(path, + return (new ContentSummaryProcessor(abfsStore, + contentSummaryExecutorService)).getContentSummary(path, tracingContext); } catch (InterruptedException e) { LOG.debug("Thread interrupted"); - throw new IOException(e); - } catch(ExecutionException ex) { + throw new InterruptedIOException(e.getMessage()); + } catch (ExecutionException ex) { LOG.debug("GetContentSummary failed with error: {}", ex.getMessage()); - throw new IOException(ex); + throw new PathIOException(path.toString(), ex); } } diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index cbe735de0fdef..dfc890be761f8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -24,7 +24,6 @@ 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; @@ -39,9 +38,7 @@ 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); @@ -49,11 +46,8 @@ public class ContentSummaryProcessor { 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 completionService = - new ExecutorCompletionService<>(executorService); + private final ExecutorService executorService; + private final CompletionService completionService; private final LinkedBlockingQueue queue = new LinkedBlockingQueue<>(); /** @@ -62,8 +56,11 @@ public class ContentSummaryProcessor { * @param abfsStore Instance of AzureBlobFileSystemStore, used to make * listStatus calls to server */ - public ContentSummaryProcessor(ListingSupport abfsStore) { + public ContentSummaryProcessor(ListingSupport abfsStore, + ExecutorService executorService) { this.abfsStore = abfsStore; + this.executorService = executorService; + completionService = new ExecutorCompletionService<>(this.executorService); } public ContentSummary getContentSummary(Path path, TracingContext tracingContext) diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java similarity index 84% rename from hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java rename to hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java index 879e1e45b350a..bf99b50b8f6ff 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/TestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java @@ -46,13 +46,13 @@ 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 { +public class ITestGetContentSummary 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_FILES_FOR_LIST_MAX_TEST =10; +// DEFAULT_AZURE_LIST_MAX_RESULTS + 10; private static final int NUM_CONCURRENT_CALLS = 8; private final String[] directories = {"/testFolder", @@ -67,7 +67,7 @@ public class TestGetContentSummary extends AbstractAbfsIntegrationTest { private final byte[] b = new byte[TEST_BUFFER_SIZE]; - public TestGetContentSummary() throws Exception { + public ITestGetContentSummary() throws Exception { new Random().nextBytes(b); } @@ -78,19 +78,17 @@ public void testFilesystemRoot() createDirectoryStructure(); int fileCount = directories.length * FILES_PER_DIRECTORY; ContentSummary contentSummary = fs.getContentSummary(new Path("/")); - verifyContentSummary(contentSummary, directories.length, fileCount, + verifyContentSummary(contentSummary, directories.length + 2, 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(); + Path filePath = path("testFile"); + try (FSDataOutputStream out = fs.create(filePath)) { + out.write(b); + } ContentSummary contentSummary = fs.getContentSummary(filePath); verifyContentSummary(contentSummary, 0, 1, TEST_BUFFER_SIZE); } @@ -98,10 +96,9 @@ public void testFileContentSummary() throws IOException { @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"); + Path testFolder = path("testFolder"); + 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); @@ -111,8 +108,8 @@ public void testLeafDir() throws IOException { public void testIntermediateDirWithFilesOnly() throws IOException, ExecutionException, InterruptedException { AzureBlobFileSystem fs = getFileSystem(); - fs.mkdirs(new Path("/testFolder")); - Path intermediateDir = new Path("/testFolder/testFolder1"); + Path testFolder = path("testFolder"); + Path intermediateDir = new Path(testFolder + "/testFolder1"); fs.mkdirs(intermediateDir); populateDirWithFiles(intermediateDir, FILES_PER_DIRECTORY); ContentSummary contentSummary = @@ -125,11 +122,10 @@ public void testIntermediateDirWithFilesOnly() public void testIntermediateDirWithFilesAndSubdirs() throws IOException, ExecutionException, InterruptedException { AzureBlobFileSystem fs = getFileSystem(); - fs.mkdirs(new Path("/testFolder")); - Path intermediateDir = new Path("/testFolder/testFolder1"); + Path intermediateDir = new Path(path("/testFolder") + "/testFolder1"); fs.mkdirs(intermediateDir); populateDirWithFiles(intermediateDir, FILES_PER_DIRECTORY); - fs.mkdirs(new Path("/testFolder/testFolder1/testFolder3")); + fs.mkdirs(new Path(intermediateDir + "/testFolder3")); fs.registerListener( new TracingHeaderValidator(getConfiguration().getClientCorrelationId(), fs.getFileSystemId(), FSOperationType.GET_CONTENT_SUMMARY, true, @@ -144,9 +140,7 @@ public void testIntermediateDirWithFilesAndSubdirs() public void testDirOverListMaxResultsItems() throws IOException, ExecutionException, InterruptedException { AzureBlobFileSystem fs = getFileSystem(); - fs.mkdirs(new Path("/testFolder")); - Path pathToListMaxDir = new Path("/testFolder/listMaxDir"); - fs.mkdirs(pathToListMaxDir); + Path pathToListMaxDir = new Path(path("/testFolder") + "/listMaxDir"); fs.mkdirs(new Path(pathToListMaxDir + "/testFolder2")); populateDirWithFiles(pathToListMaxDir, NUM_FILES_FOR_LIST_MAX_TEST); verifyContentSummary( @@ -169,10 +163,10 @@ public void testConcurrentGetContentSummaryCalls() TimeUnit.SECONDS, new SynchronousQueue<>()); CompletionService completionService = new ExecutorCompletionService<>(executorService); - createDirectoryStructure(); + Path testPath = createDirectoryStructure(); for (int i = 0; i < NUM_CONCURRENT_CALLS; i++) { completionService.submit(() -> fs.getContentSummary(new Path( - "/testFolder"))); + testPath + "/testFolder"))); } for (int i = 0; i < NUM_CONCURRENT_CALLS; i++) { ContentSummary contentSummary = completionService.take().get(); @@ -184,6 +178,9 @@ public void testConcurrentGetContentSummaryCalls() private void verifyContentSummary(ContentSummary contentSummary, long expectedDirectoryCount, long expectedFileCount, long expectedByteCount) { + System.out.println(contentSummary); + System.out.println(expectedDirectoryCount + " : " + expectedFileCount + + " : " + expectedByteCount); Assertions.assertThat(contentSummary.getDirectoryCount()) .describedAs("Incorrect directory count").isEqualTo(expectedDirectoryCount); Assertions.assertThat(contentSummary.getFileCount()) @@ -194,14 +191,16 @@ private void verifyContentSummary(ContentSummary contentSummary, .describedAs("Incorrect value of space consumed").isEqualTo(expectedByteCount); } - private void createDirectoryStructure() + private Path createDirectoryStructure() throws IOException, ExecutionException, InterruptedException { AzureBlobFileSystem fs = getFileSystem(); + Path testPath = path("testPath"); for (String directory : directories) { - Path dirPath = new Path(directory); + Path dirPath = new Path(testPath + directory); fs.mkdirs(dirPath); populateDirWithFiles(dirPath, FILES_PER_DIRECTORY); } + return testPath; } private void populateDirWithFiles(Path directory, int numFiles) @@ -218,10 +217,10 @@ private void populateDirWithFiles(Path directory, int numFiles) for (Future task : tasks) { task.get(); } - FSDataOutputStream out = getFileSystem() - .append(new Path(directory + "/test0")); - out.write(b); - out.close(); + try (FSDataOutputStream out = getFileSystem().append( + new Path(directory + "/test0"))) { + out.write(b); + } es.shutdownNow(); } } From c9d65aa0c4d5faed01e6f62abbc803739a06ae80 Mon Sep 17 00:00:00 2001 From: sumangala Date: Thu, 26 Aug 2021 23:57:11 +0530 Subject: [PATCH 29/33] review comments part 2: move executor->abfsStore --- .../fs/azurebfs/AzureBlobFileSystem.java | 6 +-- .../fs/azurebfs/AzureBlobFileSystemStore.java | 7 +++ .../services/ContentSummaryProcessor.java | 49 +++++++++++-------- .../services/ITestGetContentSummary.java | 4 +- 4 files changed, 38 insertions(+), 28 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 5008476f9c058..551140d64eea8 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -127,9 +127,6 @@ public class AzureBlobFileSystem extends FileSystem private String clientCorrelationId; private TracingHeaderFormat tracingHeaderFormat; private Listener listener; - private final ExecutorService contentSummaryExecutorService = Executors.newCachedThreadPool(); -// private final ExecutorService contentSummaryExecutorService = new ThreadPoolExecutor( -// 0, 16, 5, TimeUnit.SECONDS, new SynchronousQueue<>()); @Override public void initialize(URI uri, Configuration configuration) @@ -453,8 +450,7 @@ public ContentSummary getContentSummary(Path path) throws IOException { TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.GET_CONTENT_SUMMARY, true, tracingHeaderFormat, listener); - return (new ContentSummaryProcessor(abfsStore, - contentSummaryExecutorService)).getContentSummary(path, + return (new ContentSummaryProcessor(abfsStore)).getContentSummary(path, tracingContext); } catch (InterruptedException e) { LOG.debug("Thread interrupted"); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index de6f676bab55a..2a97a460501cf 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -51,6 +51,8 @@ import java.util.Set; import java.util.WeakHashMap; import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -166,6 +168,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private final IdentityTransformerInterface identityTransformer; private final AbfsPerfTracker abfsPerfTracker; private final AbfsCounters abfsCounters; + private final ExecutorService contentSummaryExecutorService = Executors.newCachedThreadPool(); /** * The set of directories where we should store files as append blobs. @@ -1669,6 +1672,10 @@ private AbfsPerfInfo startTracking(String callerName, String calleeName) { return new AbfsPerfInfo(abfsPerfTracker, callerName, calleeName); } + public ExecutorService getContentSummaryExecutorService() { + return contentSummaryExecutorService; + } + private static class VersionedFileStatus extends FileStatus { private final String version; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index dfc890be761f8..3b68d4cfa68e3 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -29,14 +29,20 @@ 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; +import org.apache.hadoop.fs.azurebfs.AzureBlobFileSystemStore; +import org.apache.hadoop.fs.azurebfs.utils.TracingContext; +/** + * Class to carry out parallelized recursive listing on a given path to + * collect directory and file count/size information, as part of the + * implementation for the Filesystem method getContentSummary + */ public class ContentSummaryProcessor { private static final int MAX_THREAD_COUNT = 16; private static final int POLL_TIMEOUT = 100; @@ -56,33 +62,34 @@ public class ContentSummaryProcessor { * @param abfsStore Instance of AzureBlobFileSystemStore, used to make * listStatus calls to server */ - public ContentSummaryProcessor(ListingSupport abfsStore, - ExecutorService executorService) { + public ContentSummaryProcessor(ListingSupport abfsStore) { this.abfsStore = abfsStore; - this.executorService = executorService; + this.executorService = ((AzureBlobFileSystemStore) abfsStore).getContentSummaryExecutorService(); completionService = new ExecutorCompletionService<>(this.executorService); } - 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()); - } + public ContentSummary getContentSummary(Path path, + TracingContext tracingContext) + throws IOException, ExecutionException, InterruptedException { + + 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()) + ContentSummary.Builder builder = + new ContentSummary.Builder().directoryCount( + directoryCount.get()).fileCount(fileCount.get()) .length(totalBytes.get()).spaceConsumed(totalBytes.get()); return builder.build(); } diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java index bf99b50b8f6ff..b7732e493b7b1 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java @@ -208,7 +208,7 @@ private void populateDirWithFiles(Path directory, int numFiles) final List> tasks = new ArrayList<>(); ExecutorService es = Executors.newFixedThreadPool(10); for (int i = 0; i < numFiles; i++) { - final Path fileName = new Path(directory + "/test" + i); + final Path fileName = new Path(directory, String.format("test-%02d", i)); tasks.add(es.submit(() -> { touch(fileName); return null; @@ -218,7 +218,7 @@ private void populateDirWithFiles(Path directory, int numFiles) task.get(); } try (FSDataOutputStream out = getFileSystem().append( - new Path(directory + "/test0"))) { + new Path(directory + "/test-00"))) { out.write(b); } es.shutdownNow(); From 3039f7fd7ec82599ab5cdfaaad5cf74ce71bd700 Mon Sep 17 00:00:00 2001 From: sumangala Date: Fri, 10 Sep 2021 16:31:49 +0530 Subject: [PATCH 30/33] use iterator + rejected-ex handler --- .../hadoop/fs/azurebfs/AbfsConfiguration.java | 6 ++++- .../fs/azurebfs/AzureBlobFileSystem.java | 5 ++-- .../fs/azurebfs/AzureBlobFileSystemStore.java | 18 ++++++++++++- .../AbfsListStatusRemoteIterator.java | 9 ++++--- .../services/ContentSummaryProcessor.java | 26 ++++++++++--------- .../ITestAbfsListStatusRemoteIterator.java | 26 ++++++++----------- .../services/ITestGetContentSummary.java | 6 ++--- 7 files changed, 57 insertions(+), 39 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java index 50cc57447f92b..12b746a3abdd5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AbfsConfiguration.java @@ -992,7 +992,7 @@ public ExponentialRetryPolicy getOauthTokenFetchRetryPolicy() { public int getWriteMaxConcurrentRequestCount() { if (this.writeMaxConcurrentRequestCount < 1) { - return 4 * Runtime.getRuntime().availableProcessors(); + return 4 * getAvailableProcessorCount(); } return this.writeMaxConcurrentRequestCount; } @@ -1013,6 +1013,10 @@ public String getClientProvidedEncryptionKey() { return rawConfig.get(accSpecEncKey, null); } + public static int getAvailableProcessorCount() { + return Runtime.getRuntime().availableProcessors(); + } + @VisibleForTesting void setReadBufferSize(int bufferSize) { this.readBufferSize = bufferSize; diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 551140d64eea8..a8dfc30375fc5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -1169,9 +1169,8 @@ public RemoteIterator listStatusIterator(Path path) if (abfsStore.getAbfsConfiguration().enableAbfsListIterator()) { TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.LISTSTATUS, true, tracingHeaderFormat, listener); - AbfsListStatusRemoteIterator abfsLsItr = - new AbfsListStatusRemoteIterator(getFileStatus(path, tracingContext), abfsStore, - tracingContext); + AbfsListStatusRemoteIterator abfsLsItr = new AbfsListStatusRemoteIterator( + path, abfsStore, tracingContext); return RemoteIterators.typeCastingRemoteIterator(abfsLsItr); } else { return super.listStatusIterator(path); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index 2a97a460501cf..bf54aac3eb6f5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -53,6 +53,11 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.RejectedExecutionHandler; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; @@ -168,7 +173,7 @@ public class AzureBlobFileSystemStore implements Closeable, ListingSupport { private final IdentityTransformerInterface identityTransformer; private final AbfsPerfTracker abfsPerfTracker; private final AbfsCounters abfsCounters; - private final ExecutorService contentSummaryExecutorService = Executors.newCachedThreadPool(); + private final ThreadPoolExecutor contentSummaryExecutorService; /** * The set of directories where we should store files as append blobs. @@ -239,6 +244,17 @@ public AzureBlobFileSystemStore(URI uri, boolean isSecureScheme, this.appendBlobDirSet = new HashSet<>(Arrays.asList( abfsConfiguration.getAppendBlobDirs().split(AbfsHttpConstants.COMMA))); } + contentSummaryExecutorService = new ThreadPoolExecutor(0, + 4 * AbfsConfiguration.getAvailableProcessorCount(), 60, + TimeUnit.SECONDS, new SynchronousQueue<>()); + contentSummaryExecutorService.setRejectedExecutionHandler( + (runnable, threadPoolExecutor) -> { + try { + contentSummaryExecutorService.getQueue().put(runnable); + } catch (InterruptedException e) { + LOG.debug("Could not submit GetContentSummary task to thread pool"); + } + }); } /** diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java index 835217f945af5..805abe3412e4d 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/AbfsListStatusRemoteIterator.java @@ -33,6 +33,7 @@ import org.slf4j.LoggerFactory; import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.RemoteIterator; import org.apache.hadoop.fs.azurebfs.utils.TracingContext; @@ -46,7 +47,7 @@ public class AbfsListStatusRemoteIterator private static final int MAX_QUEUE_SIZE = 10; private static final long POLL_WAIT_TIME_IN_MS = 250; - private final FileStatus fileStatus; + private final Path path; private final ListingSupport listingSupport; private final ArrayBlockingQueue iteratorsQueue; private final TracingContext tracingContext; @@ -56,9 +57,9 @@ public class AbfsListStatusRemoteIterator private String continuation; private Iterator currIterator; - public AbfsListStatusRemoteIterator(final FileStatus fileStatus, + public AbfsListStatusRemoteIterator(final Path path, final ListingSupport listingSupport, TracingContext tracingContext) { - this.fileStatus = fileStatus; + this.path = path; this.listingSupport = listingSupport; this.tracingContext = tracingContext; iteratorsQueue = new ArrayBlockingQueue<>(MAX_QUEUE_SIZE); @@ -147,7 +148,7 @@ private void addNextBatchIteratorToQueue() throws IOException, InterruptedException { List fileStatuses = new ArrayList<>(); continuation = listingSupport - .listStatus(fileStatus.getPath(), null, fileStatuses, FETCH_ALL_FALSE, + .listStatus(path, null, fileStatuses, FETCH_ALL_FALSE, continuation, tracingContext); if (!fileStatuses.isEmpty()) { iteratorsQueue.put(fileStatuses.iterator()); diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java index 3b68d4cfa68e3..b4329a99d9d14 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/services/ContentSummaryProcessor.java @@ -95,24 +95,25 @@ public ContentSummary getContentSummary(Path path, } /** - * Calls listStatus on given path and populated fileStatus queue with + * Calls listStatus on given fileStatus 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 + * under a given fileStatus + * @param fileStatus : 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) { - if (fileStatus.isDirectory()) { - queue.put(fileStatus); + private void processDirectoryTree(Path fileStatus, + TracingContext tracingContext) throws IOException, InterruptedException { + AbfsListStatusRemoteIterator iterator = new AbfsListStatusRemoteIterator( + fileStatus, abfsStore, tracingContext); + while (iterator.hasNext()) { + FileStatus status = iterator.next(); + if (status.isDirectory()) { + queue.put(status); processDirectory(); conditionalSubmitTaskToExecutor(tracingContext); } else { - processFile(fileStatus); + processFile(status); } } } @@ -141,7 +142,8 @@ private synchronized void conditionalSubmitTaskToExecutor(TracingContext tracing FileStatus fileStatus1; while ((fileStatus1 = queue.poll(POLL_TIMEOUT, TimeUnit.MILLISECONDS)) != null) { - processDirectoryTree(fileStatus1.getPath(), new TracingContext(tracingContext)); + processDirectoryTree(fileStatus1.getPath(), + new TracingContext(tracingContext)); } return null; }); diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java index 9e81a0127b6cb..91a87621eaea3 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/ITestAbfsListStatusRemoteIterator.java @@ -64,9 +64,8 @@ public void testAbfsIteratorWithHasNext() throws Exception { testDir, "testListPath"); ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore()); - RemoteIterator fsItr = new AbfsListStatusRemoteIterator( - getFileSystem().getFileStatus(testDir), listngSupport, - getTestTracingContext(getFileSystem(), true)); + RemoteIterator fsItr = new AbfsListStatusRemoteIterator(testDir, + listngSupport, getTestTracingContext(getFileSystem(), true)); Assertions.assertThat(fsItr) .describedAs("RemoteIterator should be instance of " + "AbfsListStatusRemoteIterator by default") @@ -101,12 +100,10 @@ public void testAbfsIteratorWithoutHasNext() throws Exception { final List fileNames = createFilesUnderDirectory(TEST_FILES_NUMBER, testDir, "testListPath"); - ListingSupport listngSupport = Mockito.spy(getFileSystem().getAbfsStore()); - RemoteIterator fsItr = new AbfsListStatusRemoteIterator( - getFileSystem().getFileStatus(testDir), listngSupport, - getTestTracingContext(getFileSystem(), true)); - Assertions.assertThat(fsItr) - .describedAs("RemoteIterator should be instance of " + ListingSupport listingSupport = Mockito.spy(getFileSystem().getAbfsStore()); + RemoteIterator fsItr = new AbfsListStatusRemoteIterator(testDir, + listingSupport, getTestTracingContext(getFileSystem(), true)); + Assertions.assertThat(fsItr).describedAs("RemoteIterator should be instance of " + "AbfsListStatusRemoteIterator by default") .isInstanceOf(AbfsListStatusRemoteIterator.class); int itrCount = 0; @@ -130,7 +127,7 @@ public void testAbfsIteratorWithoutHasNext() throws Exception { + "there should be no more elements in the fileNames") .isEqualTo(0); int minNumberOfInvokations = TEST_FILES_NUMBER / 10; - verify(listngSupport, Mockito.atLeast(minNumberOfInvokations)) + verify(listingSupport, Mockito.atLeast(minNumberOfInvokations)) .listStatus(any(Path.class), nullable(String.class), anyList(), anyBoolean(), nullable(String.class), @@ -209,7 +206,7 @@ public void testNextWhenNoMoreElementsPresent() throws Exception { Path testDir = createTestDirectory(); setPageSize(10); RemoteIterator fsItr = - new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir), + new AbfsListStatusRemoteIterator(testDir, getFileSystem().getAbfsStore(), getTestTracingContext(getFileSystem(), true)); fsItr = Mockito.spy(fsItr); @@ -257,9 +254,8 @@ public void testIOException() throws Exception { String exceptionMessage = "test exception"; ListingSupport lsSupport =getMockListingSupport(exceptionMessage); - RemoteIterator fsItr = - new AbfsListStatusRemoteIterator(getFileSystem().getFileStatus(testDir), - lsSupport, getTestTracingContext(getFileSystem(), true)); + RemoteIterator fsItr = new AbfsListStatusRemoteIterator(testDir, lsSupport, + getTestTracingContext(getFileSystem(), true)); Assertions.assertThatThrownBy(() -> fsItr.next()) .describedAs( @@ -270,7 +266,7 @@ public void testIOException() throws Exception { } @Test - public void testNonExistingPath() throws Throwable { + public void testNonExistingPath() { Path nonExistingDir = new Path("nonExistingPath"); Assertions.assertThatThrownBy( () -> getFileSystem().listStatusIterator(nonExistingDir)).describedAs( diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java index b7732e493b7b1..c2fe96b9dc0f0 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java @@ -51,9 +51,9 @@ public class ITestGetContentSummary 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 =10; -// DEFAULT_AZURE_LIST_MAX_RESULTS + 10; -private static final int NUM_CONCURRENT_CALLS = 8; + 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", From 8259a2e9ec9b4423d221a5c683d4e72ae9fb18aa Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 13 Sep 2021 10:05:27 +0530 Subject: [PATCH 31/33] undo extra formatting --- .../fs/azurebfs/AzureBlobFileSystem.java | 83 ++++++++++--------- .../fs/azurebfs/AzureBlobFileSystemStore.java | 3 - .../services/ITestGetContentSummary.java | 3 - 3 files changed, 45 insertions(+), 44 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index a8dfc30375fc5..3fa1e288d3389 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -143,7 +143,8 @@ public void initialize(URI uri, Configuration configuration) configuration, abfsCounters); LOG.trace("AzureBlobFileSystemStore init complete"); - final AbfsConfiguration abfsConfiguration = abfsStore.getAbfsConfiguration(); + final AbfsConfiguration abfsConfiguration = abfsStore + .getAbfsConfiguration(); clientCorrelationId = TracingContext.validateClientCorrelationID( abfsConfiguration.getClientCorrelationId()); tracingHeaderFormat = abfsConfiguration.getTracingHeaderFormat(); @@ -151,10 +152,8 @@ public void initialize(URI uri, Configuration configuration) if (abfsConfiguration.getCreateRemoteFileSystemDuringInitialization()) { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, - listener); - if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), - tracingContext) == null) { + fileSystemId, FSOperationType.CREATE_FILESYSTEM, tracingHeaderFormat, listener); + if (this.tryGetFileStatus(new Path(AbfsHttpConstants.ROOT_PATH), tracingContext) == null) { try { this.createFileSystem(tracingContext); } catch (AzureBlobFileSystemException ex) { @@ -182,7 +181,8 @@ public void initialize(URI uri, Configuration configuration) @Override public String toString() { - final StringBuilder sb = new StringBuilder("AzureBlobFileSystem{"); + final StringBuilder sb = new StringBuilder( + "AzureBlobFileSystem{"); sb.append("uri=").append(uri); sb.append(", user='").append(abfsStore.getUser()).append('\''); sb.append(", primaryUserGroup='").append(abfsStore.getPrimaryGroup()).append('\''); @@ -205,8 +205,7 @@ public void registerListener(Listener listener1) { @Override public FSDataInputStream open(final Path path, final int bufferSize) throws IOException { - LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, - bufferSize); + LOG.debug("AzureBlobFileSystem.open path: {} bufferSize: {}", path, bufferSize); // bufferSize is unused. return open(path, Optional.empty()); } @@ -219,10 +218,10 @@ private FSDataInputStream open(final Path path, try { TracingContext tracingContext = new TracingContext(clientCorrelationId, fileSystemId, FSOperationType.OPEN, tracingHeaderFormat, listener); - InputStream inputStream = abfsStore.openFileForRead(qualifiedPath, - parameters, statistics, tracingContext); + InputStream inputStream = abfsStore + .openFileForRead(qualifiedPath, parameters, statistics, tracingContext); return new FSDataInputStream(inputStream); - } catch (AzureBlobFileSystemException ex) { + } catch(AzureBlobFileSystemException ex) { checkException(path, ex); return null; } @@ -233,18 +232,21 @@ private FSDataInputStream open(final Path path, * {@link org.apache.hadoop.fs.impl.OpenFileParameters}. Ensure that * FileStatus entered is up-to-date, as it will be used to create the * InputStream (with info such as contentLength, eTag) - * * @param path The location of file to be opened * @param parameters OpenFileParameters instance; can hold FileStatus, * Configuration, bufferSize and mandatoryKeys */ @Override protected CompletableFuture openFileWithOptions( - final Path path, final OpenFileParameters parameters) throws IOException { + final Path path, final OpenFileParameters parameters) { LOG.debug("AzureBlobFileSystem.openFileWithOptions path: {}", path); AbstractFSBuilderImpl.rejectUnknownMandatoryKeys( - parameters.getMandatoryKeys(), Collections.emptySet(), "for " + path); - return LambdaUtils.eval(new CompletableFuture<>(), () -> open(path, Optional.of(parameters))); + parameters.getMandatoryKeys(), + Collections.emptySet(), + "for " + path); + return LambdaUtils.eval( + new CompletableFuture<>(), () -> + open(path, Optional.of(parameters))); } @Override @@ -253,7 +255,10 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, final long blockSize, final Progressable progress) throws IOException { LOG.debug( "AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", - f, permission, overwrite, blockSize); + f, + permission, + overwrite, + blockSize); statIncrement(CALL_CREATE); trailingPeriodCheck(f); @@ -262,8 +267,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, try { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.CREATE, overwrite, tracingHeaderFormat, - listener); + fileSystemId, FSOperationType.CREATE, overwrite, tracingHeaderFormat, listener); OutputStream outputStream = abfsStore.createFile(qualifiedPath, statistics, overwrite, permission == null ? FsPermission.getFileDefault() : permission, FsPermission.getUMask(getConf()), tracingContext); @@ -289,11 +293,11 @@ public FSDataOutputStream createNonRecursive(final Path f, final FsPermission pe final FileStatus parentFileStatus = tryGetFileStatus(parent, tracingContext); if (parentFileStatus == null) { - throw new FileNotFoundException("Cannot create file " + f.getName() + " because parent folder does not exist."); + throw new FileNotFoundException("Cannot create file " + + f.getName() + " because parent folder does not exist."); } - return create(f, permission, overwrite, bufferSize, replication, blockSize, - progress); + return create(f, permission, overwrite, bufferSize, replication, blockSize, progress); } @Override @@ -304,38 +308,40 @@ public FSDataOutputStream createNonRecursive(final Path f, final FsPermission pe // Check if file should be appended or overwritten. Assume that the file // is overwritten on if the CREATE and OVERWRITE create flags are set. - final EnumSet createflags = EnumSet.of(CreateFlag.CREATE, - CreateFlag.OVERWRITE); + final EnumSet createflags = + EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE); final boolean overwrite = flags.containsAll(createflags); // Delegate the create non-recursive call. - return this.createNonRecursive(f, permission, overwrite, bufferSize, - replication, blockSize, progress); + return this.createNonRecursive(f, permission, overwrite, + bufferSize, replication, blockSize, progress); } @Override @SuppressWarnings("deprecation") public FSDataOutputStream createNonRecursive(final Path f, - final boolean overwrite, final int bufferSize, final short replication, - final long blockSize, final Progressable progress) throws IOException { - return this.createNonRecursive(f, FsPermission.getFileDefault(), overwrite, - bufferSize, replication, blockSize, progress); + final boolean overwrite, final int bufferSize, final short replication, final long blockSize, + final Progressable progress) throws IOException { + return this.createNonRecursive(f, FsPermission.getFileDefault(), + overwrite, bufferSize, replication, blockSize, progress); } @Override public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { LOG.debug("AzureBlobFileSystem.append path: {} bufferSize: {}", - f.toString(), bufferSize); + f.toString(), + bufferSize); statIncrement(CALL_APPEND); Path qualifiedPath = makeQualified(f); try { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.APPEND, tracingHeaderFormat, listener); - OutputStream outputStream = abfsStore.openFileForWrite(qualifiedPath, - statistics, false, tracingContext); + fileSystemId, FSOperationType.APPEND, tracingHeaderFormat, + listener); + OutputStream outputStream = abfsStore + .openFileForWrite(qualifiedPath, statistics, false, tracingContext); return new FSDataOutputStream(outputStream, statistics); - } catch (AzureBlobFileSystemException ex) { + } catch(AzureBlobFileSystemException ex) { checkException(f, ex); return null; } @@ -395,7 +401,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { abfsStore.rename(qualifiedSrcPath, qualifiedDstPath, tracingContext); return true; - } catch (AzureBlobFileSystemException ex) { + } catch(AzureBlobFileSystemException ex) { LOG.debug("Rename operation failed. ", ex); checkException(src, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS, AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, @@ -409,8 +415,8 @@ public boolean rename(final Path src, final Path dst) throws IOException { @Override public boolean delete(final Path f, final boolean recursive) throws IOException { - LOG.debug("AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), - recursive); + LOG.debug( + "AzureBlobFileSystem.delete path: {} recursive: {}", f.toString(), recursive); statIncrement(CALL_DELETE); Path qualifiedPath = makeQualified(f); @@ -424,7 +430,8 @@ public boolean delete(final Path f, final boolean recursive) throws IOException try { TracingContext tracingContext = new TracingContext(clientCorrelationId, - fileSystemId, FSOperationType.DELETE, tracingHeaderFormat, listener); + fileSystemId, FSOperationType.DELETE, tracingHeaderFormat, + listener); abfsStore.delete(qualifiedPath, recursive, tracingContext); return true; } catch (AzureBlobFileSystemException ex) { diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java index bf54aac3eb6f5..30847be82ecc5 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystemStore.java @@ -52,9 +52,6 @@ import java.util.WeakHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.LinkedBlockingQueue; -import java.util.concurrent.RejectedExecutionHandler; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java index c2fe96b9dc0f0..5d6994e951d83 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azurebfs/services/ITestGetContentSummary.java @@ -178,9 +178,6 @@ public void testConcurrentGetContentSummaryCalls() private void verifyContentSummary(ContentSummary contentSummary, long expectedDirectoryCount, long expectedFileCount, long expectedByteCount) { - System.out.println(contentSummary); - System.out.println(expectedDirectoryCount + " : " + expectedFileCount + - " : " + expectedByteCount); Assertions.assertThat(contentSummary.getDirectoryCount()) .describedAs("Incorrect directory count").isEqualTo(expectedDirectoryCount); Assertions.assertThat(contentSummary.getFileCount()) From b64b492596f56f467867801e1c850a10948d00a2 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 13 Sep 2021 10:15:27 +0530 Subject: [PATCH 32/33] more formatting --- .../fs/azurebfs/AzureBlobFileSystem.java | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index 3fa1e288d3389..cae2b126bcd7c 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -232,13 +232,13 @@ private FSDataInputStream open(final Path path, * {@link org.apache.hadoop.fs.impl.OpenFileParameters}. Ensure that * FileStatus entered is up-to-date, as it will be used to create the * InputStream (with info such as contentLength, eTag) - * @param path The location of file to be opened + * @param path The location of file to be opened * @param parameters OpenFileParameters instance; can hold FileStatus, * Configuration, bufferSize and mandatoryKeys */ @Override protected CompletableFuture openFileWithOptions( - final Path path, final OpenFileParameters parameters) { + final Path path, final OpenFileParameters parameters) throws IOException { LOG.debug("AzureBlobFileSystem.openFileWithOptions path: {}", path); AbstractFSBuilderImpl.rejectUnknownMandatoryKeys( parameters.getMandatoryKeys(), @@ -253,8 +253,7 @@ protected CompletableFuture openFileWithOptions( public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize, final short replication, final long blockSize, final Progressable progress) throws IOException { - LOG.debug( - "AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", + LOG.debug("AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", f, permission, overwrite, @@ -273,7 +272,7 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, FsPermission.getUMask(getConf()), tracingContext); statIncrement(FILES_CREATED); return new FSDataOutputStream(outputStream, statistics); - } catch (AzureBlobFileSystemException ex) { + } catch(AzureBlobFileSystemException ex) { checkException(f, ex); return null; } @@ -328,7 +327,8 @@ public FSDataOutputStream createNonRecursive(final Path f, @Override public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { - LOG.debug("AzureBlobFileSystem.append path: {} bufferSize: {}", + LOG.debug( + "AzureBlobFileSystem.append path: {} bufferSize: {}", f.toString(), bufferSize); statIncrement(CALL_APPEND); @@ -364,7 +364,7 @@ public boolean rename(final Path src, final Path dst) throws IOException { fileSystemId, FSOperationType.RENAME, true, tracingHeaderFormat, listener); // rename under same folder; - if (makeQualified(parentFolder).equals(qualifiedDstPath)) { + if(makeQualified(parentFolder).equals(qualifiedDstPath)) { return tryGetFileStatus(qualifiedSrcPath, tracingContext) != null; } @@ -403,7 +403,10 @@ public boolean rename(final Path src, final Path dst) throws IOException { return true; } catch(AzureBlobFileSystemException ex) { LOG.debug("Rename operation failed. ", ex); - checkException(src, ex, AzureServiceErrorCode.PATH_ALREADY_EXISTS, + checkException( + src, + ex, + AzureServiceErrorCode.PATH_ALREADY_EXISTS, AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND, From 16d9436062f0da1fe4d074a011c2947697d72240 Mon Sep 17 00:00:00 2001 From: sumangala Date: Mon, 13 Sep 2021 10:22:18 +0530 Subject: [PATCH 33/33] format --- .../fs/azurebfs/AzureBlobFileSystem.java | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java index cae2b126bcd7c..190a7a90f7394 100644 --- a/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java +++ b/hadoop-tools/hadoop-azure/src/main/java/org/apache/hadoop/fs/azurebfs/AzureBlobFileSystem.java @@ -250,9 +250,8 @@ protected CompletableFuture openFileWithOptions( } @Override - public FSDataOutputStream create(final Path f, final FsPermission permission, - final boolean overwrite, final int bufferSize, final short replication, - final long blockSize, final Progressable progress) throws IOException { + public FSDataOutputStream create(final Path f, final FsPermission permission, final boolean overwrite, final int bufferSize, + final short replication, final long blockSize, final Progressable progress) throws IOException { LOG.debug("AzureBlobFileSystem.create path: {} permission: {} overwrite: {} bufferSize: {}", f, permission, @@ -404,13 +403,14 @@ public boolean rename(final Path src, final Path dst) throws IOException { } catch(AzureBlobFileSystemException ex) { LOG.debug("Rename operation failed. ", ex); checkException( - src, - ex, - AzureServiceErrorCode.PATH_ALREADY_EXISTS, - AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, - AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, - AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND, - AzureServiceErrorCode.INTERNAL_OPERATION_ABORT); + src, + ex, + AzureServiceErrorCode.PATH_ALREADY_EXISTS, + AzureServiceErrorCode.INVALID_RENAME_SOURCE_PATH, + AzureServiceErrorCode.SOURCE_PATH_NOT_FOUND, + AzureServiceErrorCode.INVALID_SOURCE_OR_DESTINATION_RESOURCE_TYPE, + AzureServiceErrorCode.RENAME_DESTINATION_PARENT_PATH_NOT_FOUND, + AzureServiceErrorCode.INTERNAL_OPERATION_ABORT); return false; }