From 4b5b54c73f2fd9146237087a59453e2b5d70f9ed Mon Sep 17 00:00:00 2001 From: Thomas Marquardt Date: Wed, 24 Jun 2020 18:37:25 +0000 Subject: [PATCH] HADOOP-17089: WASB: Update azure-storage-java SDK Contributed by Thomas Marquardt DETAILS: WASB depends on the Azure Storage Java SDK. There is a concurrency bug in the Azure Storage Java SDK that can cause the results of a list blobs operation to appear empty. This causes the Filesystem listStatus and similar APIs to return empty results. This has been seen in Spark work loads when jobs use more than one executor core. See Azure/azure-storage-java#546 for details on the bug in the Azure Storage SDK. TESTS: A new test was added to validate the fix. All tests are passing: wasb: mvn -T 1C -Dparallel-tests=wasb -Dscale -DtestsThreadCount=8 clean verify Tests run: 248, Failures: 0, Errors: 0, Skipped: 11 Tests run: 651, Failures: 0, Errors: 0, Skipped: 65 abfs: mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount=8 clean verify Tests run: 64, Failures: 0, Errors: 0, Skipped: 0 Tests run: 437, Failures: 0, Errors: 0, Skipped: 33 Tests run: 206, Failures: 0, Errors: 0, Skipped: 24 --- hadoop-project/pom.xml | 2 +- ...tNativeAzureFileSystemConcurrencyLive.java | 59 ++++++++++++++++++- 2 files changed, 58 insertions(+), 3 deletions(-) diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index 48928b508e3..4e819cd896a 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -1419,7 +1419,7 @@ com.microsoft.azure azure-storage - 7.0.0 + 7.0.1 diff --git a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java index 1c868ea0ff1..2c99b84394f 100644 --- a/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java +++ b/hadoop-tools/hadoop-azure/src/test/java/org/apache/hadoop/fs/azure/ITestNativeAzureFileSystemConcurrencyLive.java @@ -20,6 +20,7 @@ package org.apache.hadoop.fs.azure; import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.junit.Assert; @@ -130,15 +131,56 @@ public class ITestNativeAzureFileSystemConcurrencyLive } } + /** + * Validate the bug fix for HADOOP-17089. Please note that we were never + * able to reproduce this except during a Spark job that ran for multiple days + * and in a hacked-up azure-storage SDK that added sleep before and after + * the call to factory.setNamespaceAware(true) as shown in the description of + * + * @see https://github.com/Azure/azure-storage-java/pull/546 + */ + @Test(timeout = TEST_EXECUTION_TIMEOUT) + public void testConcurrentList() throws Exception { + final Path testDir = new Path("/tmp/data-loss/11230174258112/_temporary/0/_temporary/attempt_20200624190514_0006_m_0"); + final Path testFile = new Path(testDir, "part-00004-15ea87b1-312c-4fdf-1820-95afb3dfc1c3-a010.snappy.parquet"); + fs.create(testFile).close(); + List tasks = new ArrayList<>(THREAD_COUNT); + + for (int i = 0; i < THREAD_COUNT; i++) { + tasks.add(new ListTask(fs, testDir)); + } + + ExecutorService es = null; + try { + es = Executors.newFixedThreadPool(THREAD_COUNT); + + List> futures = es.invokeAll(tasks); + + for (Future future : futures) { + Assert.assertTrue(future.isDone()); + + // we are using Callable, so if an exception + // occurred during the operation, it will be thrown + // when we call get + long fileCount = future.get(); + assertEquals("The list should always contain 1 file.", 1, fileCount); + } + } finally { + if (es != null) { + es.shutdownNow(); + } + } + } + abstract class FileSystemTask implements Callable { private final FileSystem fileSystem; private final Path path; - protected FileSystem getFileSystem() { + FileSystem getFileSystem() { return this.fileSystem; } - protected Path getFilePath() { + Path getFilePath() { return this.path; } @@ -182,4 +224,17 @@ public class ITestNativeAzureFileSystemConcurrencyLive return null; } } + + class ListTask extends FileSystemTask { + ListTask(FileSystem fs, Path p) { + super(fs, p); + } + + public Integer call() throws Exception { + FileSystem fs = getFileSystem(); + Path p = getFilePath(); + FileStatus[] files = fs.listStatus(p); + return files.length; + } + } }