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
This commit is contained in:
parent
84110d850e
commit
4b5b54c73f
|
@ -1419,7 +1419,7 @@
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>com.microsoft.azure</groupId>
|
<groupId>com.microsoft.azure</groupId>
|
||||||
<artifactId>azure-storage</artifactId>
|
<artifactId>azure-storage</artifactId>
|
||||||
<version>7.0.0</version>
|
<version>7.0.1</version>
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<!--Wildfly openssl dependency is introduced by HADOOP-15669-->
|
<!--Wildfly openssl dependency is introduced by HADOOP-15669-->
|
||||||
|
|
|
@ -20,6 +20,7 @@ package org.apache.hadoop.fs.azure;
|
||||||
|
|
||||||
|
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
import org.apache.hadoop.fs.FileSystem;
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.junit.Assert;
|
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 <a href="https://github.com/Azure/azure-storage-java/pull/546">https://github.com/Azure/azure-storage-java/pull/546</a>
|
||||||
|
*/
|
||||||
|
@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<ListTask> 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<Future<Integer>> futures = es.invokeAll(tasks);
|
||||||
|
|
||||||
|
for (Future<Integer> future : futures) {
|
||||||
|
Assert.assertTrue(future.isDone());
|
||||||
|
|
||||||
|
// we are using Callable<V>, 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<V> implements Callable<V> {
|
abstract class FileSystemTask<V> implements Callable<V> {
|
||||||
private final FileSystem fileSystem;
|
private final FileSystem fileSystem;
|
||||||
private final Path path;
|
private final Path path;
|
||||||
|
|
||||||
protected FileSystem getFileSystem() {
|
FileSystem getFileSystem() {
|
||||||
return this.fileSystem;
|
return this.fileSystem;
|
||||||
}
|
}
|
||||||
|
|
||||||
protected Path getFilePath() {
|
Path getFilePath() {
|
||||||
return this.path;
|
return this.path;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -182,4 +224,17 @@ public class ITestNativeAzureFileSystemConcurrencyLive
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class ListTask extends FileSystemTask<Integer> {
|
||||||
|
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;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue