Skip to content

Commit

Permalink
HADOOP-17089: WASB: Update azure-storage-java SDK
Browse files Browse the repository at this point in the history
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:

$mvn -T 1C -Dparallel-tests=wasb -Dscale -DtestsThreadCount=8 clean verify
Tests run: 231, Failures: 0, Errors: 0, Skipped: 4
Tests run: 588, Failures: 0, Errors: 0, Skipped: 12
Tests run: 3, Failures: 0, Errors: 0, Skipped: 0

$mvn -T 1C -Dparallel-tests=abfs -Dscale -DtestsThreadCount=8 clean verify
Tests run: 37, Failures: 0, Errors: 0, Skipped: 0
Tests run: 407, Failures: 0, Errors: 0, Skipped: 34
Tests run: 151, Failures: 0, Errors: 0, Skipped: 19
Tests run: 206, Failures: 0, Errors: 0, Skipped: 24
  • Loading branch information
ThomasMarquardt committed Jun 26, 2020
1 parent e81002b commit 0d4f9c7
Show file tree
Hide file tree
Showing 2 changed files with 58 additions and 8 deletions.
7 changes: 1 addition & 6 deletions hadoop-project/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -1131,7 +1131,7 @@
<dependency>
<groupId>com.microsoft.azure</groupId>
<artifactId>azure-storage</artifactId>
<version>7.0.0</version>
<version>7.0.1</version>
</dependency>

<!--Wildfly openssl dependency is introduced by HADOOP-15669-->
Expand Down Expand Up @@ -1189,11 +1189,6 @@
<version>1.46</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.microsoft.azure</groupId>
<artifactId>azure-storage</artifactId>
<version>5.4.0</version>
</dependency>

<dependency>
<groupId>joda-time</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@


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;
Expand Down Expand Up @@ -130,15 +131,56 @@ public void testConcurrentDeleteFile() throws Exception {
}
}

/**
* 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> {
private final FileSystem fileSystem;
private final Path path;

protected FileSystem getFileSystem() {
FileSystem getFileSystem() {
return this.fileSystem;
}

protected Path getFilePath() {
Path getFilePath() {
return this.path;
}

Expand Down Expand Up @@ -182,4 +224,17 @@ public Void call() throws Exception {
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;
}
}
}

0 comments on commit 0d4f9c7

Please sign in to comment.