HADOOP-10840. Fix OutOfMemoryError caused by metrics system in Azure File System. Contributed by Shanyu Zhao.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1611247 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Chris Nauroth 2014-07-17 05:04:53 +00:00
parent 3c193811ca
commit 0a02b5a19b
5 changed files with 43 additions and 17 deletions

View File

@ -384,6 +384,9 @@ Trunk (Unreleased)
HADOOP-10816. KeyShell returns -1 on error to the shell, should be 1. HADOOP-10816. KeyShell returns -1 on error to the shell, should be 1.
(Mike Yoder via wang) (Mike Yoder via wang)
HADOOP-10840. Fix OutOfMemoryError caused by metrics system in Azure File
System. (Shanyu Zhao via cnauroth)
OPTIMIZATIONS OPTIMIZATIONS
HADOOP-7761. Improve the performance of raw comparisons. (todd) HADOOP-7761. Improve the performance of raw comparisons. (todd)

View File

@ -373,6 +373,8 @@ public class NativeAzureFileSystem extends FileSystem {
private Path workingDir; private Path workingDir;
private long blockSize = MAX_AZURE_BLOCK_SIZE; private long blockSize = MAX_AZURE_BLOCK_SIZE;
private AzureFileSystemInstrumentation instrumentation; private AzureFileSystemInstrumentation instrumentation;
private String metricsSourceName;
private boolean isClosed = false;
private static boolean suppressRetryPolicy = false; private static boolean suppressRetryPolicy = false;
// A counter to create unique (within-process) names for my metrics sources. // A counter to create unique (within-process) names for my metrics sources.
private static AtomicInteger metricsSourceNameCounter = new AtomicInteger(); private static AtomicInteger metricsSourceNameCounter = new AtomicInteger();
@ -482,11 +484,10 @@ public class NativeAzureFileSystem extends FileSystem {
// Make sure the metrics system is available before interacting with Azure // Make sure the metrics system is available before interacting with Azure
AzureFileSystemMetricsSystem.fileSystemStarted(); AzureFileSystemMetricsSystem.fileSystemStarted();
String sourceName = newMetricsSourceName(), metricsSourceName = newMetricsSourceName();
sourceDesc = "Azure Storage Volume File System metrics"; String sourceDesc = "Azure Storage Volume File System metrics";
instrumentation = DefaultMetricsSystem.instance().register(sourceName, instrumentation = new AzureFileSystemInstrumentation(conf);
sourceDesc, new AzureFileSystemInstrumentation(conf)); AzureFileSystemMetricsSystem.registerSource(metricsSourceName, sourceDesc,
AzureFileSystemMetricsSystem.registerSource(sourceName, sourceDesc,
instrumentation); instrumentation);
store.initialize(uri, conf, instrumentation); store.initialize(uri, conf, instrumentation);
@ -502,7 +503,6 @@ public class NativeAzureFileSystem extends FileSystem {
LOG.debug(" blockSize = " LOG.debug(" blockSize = "
+ conf.getLong(AZURE_BLOCK_SIZE_PROPERTY_NAME, MAX_AZURE_BLOCK_SIZE)); + conf.getLong(AZURE_BLOCK_SIZE_PROPERTY_NAME, MAX_AZURE_BLOCK_SIZE));
} }
} }
private NativeFileSystemStore createDefaultStore(Configuration conf) { private NativeFileSystemStore createDefaultStore(Configuration conf) {
@ -1337,7 +1337,11 @@ public class NativeAzureFileSystem extends FileSystem {
} }
@Override @Override
public void close() throws IOException { public synchronized void close() throws IOException {
if (isClosed) {
return;
}
// Call the base close() to close any resources there. // Call the base close() to close any resources there.
super.close(); super.close();
// Close the store // Close the store
@ -1349,12 +1353,14 @@ public class NativeAzureFileSystem extends FileSystem {
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
AzureFileSystemMetricsSystem.unregisterSource(metricsSourceName);
AzureFileSystemMetricsSystem.fileSystemClosed(); AzureFileSystemMetricsSystem.fileSystemClosed();
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug("Submitting metrics when file system closed took " LOG.debug("Submitting metrics when file system closed took "
+ (System.currentTimeMillis() - startTime) + " ms."); + (System.currentTimeMillis() - startTime) + " ms.");
} }
isClosed = true;
} }
/** /**
@ -1498,6 +1504,13 @@ public class NativeAzureFileSystem extends FileSystem {
handleFilesWithDanglingTempData(root, new DanglingFileDeleter()); handleFilesWithDanglingTempData(root, new DanglingFileDeleter());
} }
@Override
protected void finalize() throws Throwable {
LOG.debug("finalize() called.");
close();
super.finalize();
}
/** /**
* Encode the key with a random prefix for load balancing in Azure storage. * Encode the key with a random prefix for load balancing in Azure storage.
* Upload data to a random temporary file then do storage side renaming to * Upload data to a random temporary file then do storage side renaming to

View File

@ -44,10 +44,8 @@ public final class AzureFileSystemMetricsSystem {
} }
public static synchronized void fileSystemClosed() { public static synchronized void fileSystemClosed() {
if (instance != null) {
instance.publishMetricsNow();
}
if (numFileSystems == 1) { if (numFileSystems == 1) {
instance.publishMetricsNow();
instance.stop(); instance.stop();
instance.shutdown(); instance.shutdown();
instance = null; instance = null;
@ -57,8 +55,15 @@ public final class AzureFileSystemMetricsSystem {
public static void registerSource(String name, String desc, public static void registerSource(String name, String desc,
MetricsSource source) { MetricsSource source) {
// Register the source with the name appended with -WasbSystem //caller has to use unique name to register source
// so that the name is globally unique. instance.register(name, desc, source);
instance.register(name + "-WasbSystem", desc, source); }
public static synchronized void unregisterSource(String name) {
if (instance != null) {
//publish metrics before unregister a metrics source
instance.publishMetricsNow();
instance.unregisterSource(name);
}
} }
} }

View File

@ -324,9 +324,7 @@ public final class AzureBlobStorageTestAccount {
String sourceName = NativeAzureFileSystem.newMetricsSourceName(); String sourceName = NativeAzureFileSystem.newMetricsSourceName();
String sourceDesc = "Azure Storage Volume File System metrics"; String sourceDesc = "Azure Storage Volume File System metrics";
AzureFileSystemInstrumentation instrumentation = AzureFileSystemInstrumentation instrumentation = new AzureFileSystemInstrumentation(conf);
DefaultMetricsSystem.instance().register(sourceName,
sourceDesc, new AzureFileSystemInstrumentation(conf));
AzureFileSystemMetricsSystem.registerSource( AzureFileSystemMetricsSystem.registerSource(
sourceName, sourceDesc, instrumentation); sourceName, sourceDesc, instrumentation);

View File

@ -516,6 +516,13 @@ public abstract class NativeAzureFileSystemBaseTest {
assertNotNull(status); assertNotNull(status);
} }
@Test
public void testCloseFileSystemTwice() throws Exception {
//make sure close() can be called multiple times without doing any harm
fs.close();
fs.close();
}
private boolean testModifiedTime(Path testPath, long time) throws Exception { private boolean testModifiedTime(Path testPath, long time) throws Exception {
FileStatus fileStatus = fs.getFileStatus(testPath); FileStatus fileStatus = fs.getFileStatus(testPath);
final long errorMargin = modifiedTimeErrorMargin; final long errorMargin = modifiedTimeErrorMargin;