HADOOP-11248. Add hadoop configuration to disable Azure Filesystem metrics collection. Contributed by Shanyu Zhao.
This commit is contained in:
parent
ec6cbece8e
commit
734eeb4f35
|
@ -339,6 +339,9 @@ Trunk (Unreleased)
|
|||
HADOOP-11022. User replaced functions get lost 2-3 levels deep (e.g.,
|
||||
sbin) (aw)
|
||||
|
||||
HADOOP-11248. Add hadoop configuration to disable Azure Filesystem metrics
|
||||
collection. (Shanyu Zhao via cnauroth)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HADOOP-7761. Improve the performance of raw comparisons. (todd)
|
||||
|
|
|
@ -641,6 +641,8 @@ public class NativeAzureFileSystem extends FileSystem {
|
|||
static final String AZURE_OUTPUT_STREAM_BUFFER_SIZE_PROPERTY_NAME =
|
||||
"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 InputStream in;
|
||||
private final String key;
|
||||
|
@ -1035,13 +1037,15 @@ public class NativeAzureFileSystem extends FileSystem {
|
|||
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);
|
||||
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);
|
||||
}
|
||||
|
||||
store.initialize(uri, conf, instrumentation);
|
||||
setConf(conf);
|
||||
|
@ -2207,8 +2211,10 @@ public class NativeAzureFileSystem extends FileSystem {
|
|||
|
||||
long startTime = System.currentTimeMillis();
|
||||
|
||||
AzureFileSystemMetricsSystem.unregisterSource(metricsSourceName);
|
||||
AzureFileSystemMetricsSystem.fileSystemClosed();
|
||||
if(!getConf().getBoolean(SKIP_AZURE_METRICS_PROPERTY_NAME, false)) {
|
||||
AzureFileSystemMetricsSystem.unregisterSource(metricsSourceName);
|
||||
AzureFileSystemMetricsSystem.fileSystemClosed();
|
||||
}
|
||||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Submitting metrics when file system closed took "
|
||||
|
|
|
@ -74,4 +74,15 @@ public class TestNativeAzureFileSystemMetricsSystem {
|
|||
assertTrue(name2.startsWith("AzureFileSystemMetrics"));
|
||||
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));
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue