HADOOP-11248. Add hadoop configuration to disable Azure Filesystem metrics collection. Contributed by Shanyu Zhao.

(cherry picked from commit 734eeb4f35)

Conflicts:
	hadoop-common-project/hadoop-common/CHANGES.txt
This commit is contained in:
cnauroth 2014-11-03 14:29:18 -08:00
parent 1646cc9f29
commit 535d98bbb3
3 changed files with 27 additions and 7 deletions

View File

@ -71,6 +71,9 @@ Release 2.7.0 - UNRELEASED
HADOOP-10840. Fix OutOfMemoryError caused by metrics system in Azure File HADOOP-10840. Fix OutOfMemoryError caused by metrics system in Azure File
System. (Shanyu Zhao via cnauroth) System. (Shanyu Zhao via cnauroth)
HADOOP-11248. Add hadoop configuration to disable Azure Filesystem metrics
collection. (Shanyu Zhao via cnauroth)
OPTIMIZATIONS OPTIMIZATIONS
HADOOP-11323. WritableComparator#compare keeps reference to byte array. HADOOP-11323. WritableComparator#compare keeps reference to byte array.

View File

@ -641,6 +641,8 @@ public class NativeAzureFileSystem extends FileSystem {
static final String AZURE_OUTPUT_STREAM_BUFFER_SIZE_PROPERTY_NAME = static final String AZURE_OUTPUT_STREAM_BUFFER_SIZE_PROPERTY_NAME =
"fs.azure.output.stream.buffer.size"; "fs.azure.output.stream.buffer.size";
public static final String SKIP_AZURE_METRICS_PROPERTY_NAME = "fs.azure.skip.metrics";
private class NativeAzureFsInputStream extends FSInputStream { private class NativeAzureFsInputStream extends FSInputStream {
private InputStream in; private InputStream in;
private final String key; private final String key;
@ -1035,13 +1037,15 @@ public class NativeAzureFileSystem extends FileSystem {
store = createDefaultStore(conf); store = createDefaultStore(conf);
} }
// Make sure the metrics system is available before interacting with Azure
AzureFileSystemMetricsSystem.fileSystemStarted();
metricsSourceName = newMetricsSourceName();
String sourceDesc = "Azure Storage Volume File System metrics";
instrumentation = new AzureFileSystemInstrumentation(conf); instrumentation = new AzureFileSystemInstrumentation(conf);
AzureFileSystemMetricsSystem.registerSource(metricsSourceName, sourceDesc, if(!conf.getBoolean(SKIP_AZURE_METRICS_PROPERTY_NAME, false)) {
// Make sure the metrics system is available before interacting with Azure
AzureFileSystemMetricsSystem.fileSystemStarted();
metricsSourceName = newMetricsSourceName();
String sourceDesc = "Azure Storage Volume File System metrics";
AzureFileSystemMetricsSystem.registerSource(metricsSourceName, sourceDesc,
instrumentation); instrumentation);
}
store.initialize(uri, conf, instrumentation); store.initialize(uri, conf, instrumentation);
setConf(conf); setConf(conf);
@ -2207,8 +2211,10 @@ public class NativeAzureFileSystem extends FileSystem {
long startTime = System.currentTimeMillis(); long startTime = System.currentTimeMillis();
AzureFileSystemMetricsSystem.unregisterSource(metricsSourceName); if(!getConf().getBoolean(SKIP_AZURE_METRICS_PROPERTY_NAME, false)) {
AzureFileSystemMetricsSystem.fileSystemClosed(); AzureFileSystemMetricsSystem.unregisterSource(metricsSourceName);
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 "

View File

@ -74,4 +74,15 @@ public class TestNativeAzureFileSystemMetricsSystem {
assertTrue(name2.startsWith("AzureFileSystemMetrics")); assertTrue(name2.startsWith("AzureFileSystemMetrics"));
assertTrue(!name1.equals(name2)); assertTrue(!name1.equals(name2));
} }
@Test
public void testSkipMetricsCollection() throws Exception {
AzureBlobStorageTestAccount a;
a = AzureBlobStorageTestAccount.createMock();
a.getFileSystem().getConf().setBoolean(
NativeAzureFileSystem.SKIP_AZURE_METRICS_PROPERTY_NAME, true);
a.getFileSystem().create(new Path("/foo")).close();
a.closeFileSystem(); // Causes the file system to close, which publishes metrics
assertEquals(0, getFilesCreated(a));
}
} }