HDDS-1089. Disable OzoneFSStorageStatistics for hadoop versions older than 2.8.

Contributed by Elek, Marton.
This commit is contained in:
Anu Engineer 2019-02-24 13:53:38 -08:00
parent d82014b70a
commit 236b105e41
3 changed files with 135 additions and 31 deletions

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.fs.ozone; package org.apache.hadoop.fs.ozone;
import java.io.IOException; import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.net.MalformedURLException; import java.net.MalformedURLException;
import java.net.URL; import java.net.URL;
import java.util.ArrayList; import java.util.ArrayList;
@ -42,8 +43,35 @@ private OzoneClientAdapterFactory() {
@SuppressFBWarnings("DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED") @SuppressFBWarnings("DP_CREATE_CLASSLOADER_INSIDE_DO_PRIVILEGED")
public static OzoneClientAdapter createAdapter( public static OzoneClientAdapter createAdapter(
String volumeStr, String volumeStr,
String bucketStr, OzoneFSStorageStatistics storageStatistics) String bucketStr) throws IOException {
return createAdapter(volumeStr, bucketStr,
(aClass) -> (OzoneClientAdapter) aClass
.getConstructor(String.class, String.class)
.newInstance(
volumeStr,
bucketStr));
}
public static OzoneClientAdapter createAdapter(
String volumeStr,
String bucketStr,
OzoneFSStorageStatistics storageStatistics)
throws IOException { throws IOException {
return createAdapter(volumeStr, bucketStr,
(aClass) -> (OzoneClientAdapter) aClass
.getConstructor(String.class, String.class,
OzoneFSStorageStatistics.class)
.newInstance(
volumeStr,
bucketStr,
storageStatistics));
}
public static OzoneClientAdapter createAdapter(
String volumeStr,
String bucketStr,
OzoneClientAdapterCreator creator) throws IOException {
ClassLoader currentClassLoader = OzoneFileSystem.class.getClassLoader(); ClassLoader currentClassLoader = OzoneFileSystem.class.getClassLoader();
List<URL> urls = new ArrayList<>(); List<URL> urls = new ArrayList<>();
@ -70,13 +98,10 @@ public static OzoneClientAdapter createAdapter(
reflectionUtils.getMethod("getClassByName", String.class) reflectionUtils.getMethod("getClassByName", String.class)
.invoke(null, "org.apache.ratis.grpc.GrpcFactory"); .invoke(null, "org.apache.ratis.grpc.GrpcFactory");
OzoneClientAdapter ozoneClientAdapter = (OzoneClientAdapter) classLoader Class<?> aClass = classLoader
.loadClass("org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl") .loadClass("org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl");
.getConstructor(String.class, String.class, OzoneClientAdapter ozoneClientAdapter =
OzoneFSStorageStatistics.class) creator.createOzoneClientAdapter(aClass);
.newInstance(
volumeStr,
bucketStr, storageStatistics);
Thread.currentThread().setContextClassLoader(contextClassLoader); Thread.currentThread().setContextClassLoader(contextClassLoader);
@ -119,4 +144,14 @@ private static void findEmbeddedLibsUrl(List<URL> urls,
} }
/**
* Interface to create OzoneClientAdapter implementation with reflection.
*/
@FunctionalInterface
interface OzoneClientAdapterCreator {
OzoneClientAdapter createOzoneClientAdapter(Class<?> clientAdapter)
throws NoSuchMethodException, IllegalAccessException,
InvocationTargetException, InstantiationException;
}
} }

View File

@ -54,11 +54,33 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
private ReplicationFactor replicationFactor; private ReplicationFactor replicationFactor;
private OzoneFSStorageStatistics storageStatistics; private OzoneFSStorageStatistics storageStatistics;
/**
* Create new OzoneClientAdapter implementation.
*
* @param volumeStr Name of the volume to use.
* @param bucketStr Name of the bucket to use
* @param storageStatistics Storage statistic (optional, can be null)
* @throws IOException In case of a problem.
*/
public OzoneClientAdapterImpl(String volumeStr, String bucketStr, public OzoneClientAdapterImpl(String volumeStr, String bucketStr,
OzoneFSStorageStatistics storageStatistics) throws IOException { OzoneFSStorageStatistics storageStatistics) throws IOException {
this(createConf(), volumeStr, bucketStr, storageStatistics); this(createConf(), volumeStr, bucketStr, storageStatistics);
} }
/**
* Create new OzoneClientAdapter implementation.
*
* @param volumeStr Name of the volume to use.
* @param bucketStr Name of the bucket to use
* @throws IOException In case of a problem.
*/
public OzoneClientAdapterImpl(String volumeStr, String bucketStr)
throws IOException {
this(createConf(), volumeStr, bucketStr, null);
}
private static OzoneConfiguration createConf() { private static OzoneConfiguration createConf() {
ClassLoader contextClassLoader = ClassLoader contextClassLoader =
Thread.currentThread().getContextClassLoader(); Thread.currentThread().getContextClassLoader();
@ -102,13 +124,17 @@ public void close() throws IOException {
@Override @Override
public InputStream createInputStream(String key) throws IOException { public InputStream createInputStream(String key) throws IOException {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.OBJECTS_READ, 1); storageStatistics.incrementCounter(Statistic.OBJECTS_READ, 1);
}
return bucket.readKey(key).getInputStream(); return bucket.readKey(key).getInputStream();
} }
@Override @Override
public OzoneFSOutputStream createKey(String key) throws IOException { public OzoneFSOutputStream createKey(String key) throws IOException {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.OBJECTS_CREATED, 1); storageStatistics.incrementCounter(Statistic.OBJECTS_CREATED, 1);
}
OzoneOutputStream ozoneOutputStream = OzoneOutputStream ozoneOutputStream =
bucket.createKey(key, 0, replicationType, replicationFactor, bucket.createKey(key, 0, replicationType, replicationFactor,
new HashMap<>()); new HashMap<>());
@ -117,7 +143,9 @@ public OzoneFSOutputStream createKey(String key) throws IOException {
@Override @Override
public void renameKey(String key, String newKeyName) throws IOException { public void renameKey(String key, String newKeyName) throws IOException {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.OBJECTS_RENAMED, 1); storageStatistics.incrementCounter(Statistic.OBJECTS_RENAMED, 1);
}
bucket.renameKey(key, newKeyName); bucket.renameKey(key, newKeyName);
} }
@ -130,7 +158,9 @@ public void renameKey(String key, String newKeyName) throws IOException {
@Override @Override
public BasicKeyInfo getKeyInfo(String keyName) { public BasicKeyInfo getKeyInfo(String keyName) {
try { try {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.OBJECTS_QUERY, 1); storageStatistics.incrementCounter(Statistic.OBJECTS_QUERY, 1);
}
OzoneKey key = bucket.getKey(keyName); OzoneKey key = bucket.getKey(keyName);
return new BasicKeyInfo( return new BasicKeyInfo(
keyName, keyName,
@ -167,7 +197,9 @@ public boolean isDirectory(BasicKeyInfo key) {
public boolean createDirectory(String keyName) { public boolean createDirectory(String keyName) {
try { try {
LOG.trace("creating dir for key:{}", keyName); LOG.trace("creating dir for key:{}", keyName);
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.OBJECTS_CREATED, 1); storageStatistics.incrementCounter(Statistic.OBJECTS_CREATED, 1);
}
bucket.createKey(keyName, 0, replicationType, replicationFactor, bucket.createKey(keyName, 0, replicationType, replicationFactor,
new HashMap<>()).close(); new HashMap<>()).close();
return true; return true;
@ -187,7 +219,9 @@ public boolean createDirectory(String keyName) {
public boolean deleteObject(String keyName) { public boolean deleteObject(String keyName) {
LOG.trace("issuing delete for key" + keyName); LOG.trace("issuing delete for key" + keyName);
try { try {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.OBJECTS_DELETED, 1); storageStatistics.incrementCounter(Statistic.OBJECTS_DELETED, 1);
}
bucket.deleteKey(keyName); bucket.deleteKey(keyName);
return true; return true;
} catch (IOException ioe) { } catch (IOException ioe) {
@ -203,13 +237,17 @@ public long getCreationTime() {
@Override @Override
public boolean hasNextKey(String key) { public boolean hasNextKey(String key) {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.OBJECTS_LIST, 1); storageStatistics.incrementCounter(Statistic.OBJECTS_LIST, 1);
}
return bucket.listKeys(key).hasNext(); return bucket.listKeys(key).hasNext();
} }
@Override @Override
public Iterator<BasicKeyInfo> listKeys(String pathKey) { public Iterator<BasicKeyInfo> listKeys(String pathKey) {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.OBJECTS_LIST, 1); storageStatistics.incrementCounter(Statistic.OBJECTS_LIST, 1);
}
return new IteratorAdapter(bucket.listKeys(pathKey)); return new IteratorAdapter(bucket.listKeys(pathKey));
} }

View File

@ -125,14 +125,30 @@ public void initialize(URI name, Configuration conf) throws IOException {
boolean isolatedClassloader = boolean isolatedClassloader =
conf.getBoolean("ozone.fs.isolated-classloader", defaultValue); conf.getBoolean("ozone.fs.isolated-classloader", defaultValue);
try {
//register only to the GlobalStorageStatistics if the class exists.
//This is required to support hadoop versions <2.7
Class.forName("org.apache.hadoop.fs.GlobalStorageStatistics");
storageStatistics = (OzoneFSStorageStatistics) storageStatistics = (OzoneFSStorageStatistics)
GlobalStorageStatistics.INSTANCE GlobalStorageStatistics.INSTANCE
.put(OzoneFSStorageStatistics.NAME, .put(OzoneFSStorageStatistics.NAME,
OzoneFSStorageStatistics::new); OzoneFSStorageStatistics::new);
} catch (ClassNotFoundException e) {
//we don't support storage statistics for hadoop2.7 and older
}
if (isolatedClassloader) { if (isolatedClassloader) {
try {
//register only to the GlobalStorageStatistics if the class exists.
//This is required to support hadoop versions <2.7
Class.forName("org.apache.hadoop.fs.GlobalStorageStatistics");
this.adapter = this.adapter =
OzoneClientAdapterFactory.createAdapter(volumeStr, bucketStr, OzoneClientAdapterFactory.createAdapter(volumeStr, bucketStr,
storageStatistics); storageStatistics);
} catch (ClassNotFoundException e) {
this.adapter =
OzoneClientAdapterFactory.createAdapter(volumeStr, bucketStr);
}
} else { } else {
OzoneConfiguration ozoneConfiguration; OzoneConfiguration ozoneConfiguration;
if (conf instanceof OzoneConfiguration) { if (conf instanceof OzoneConfiguration) {
@ -188,7 +204,9 @@ OzoneFSStorageStatistics getOzoneFSOpsCountStatistics() {
@Override @Override
public FSDataInputStream open(Path f, int bufferSize) throws IOException { public FSDataInputStream open(Path f, int bufferSize) throws IOException {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.INVOCATION_OPEN, 1); storageStatistics.incrementCounter(Statistic.INVOCATION_OPEN, 1);
}
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
LOG.trace("open() path:{}", f); LOG.trace("open() path:{}", f);
final FileStatus fileStatus = getFileStatus(f); final FileStatus fileStatus = getFileStatus(f);
@ -207,7 +225,9 @@ public FSDataOutputStream create(Path f, FsPermission permission,
short replication, long blockSize, short replication, long blockSize,
Progressable progress) throws IOException { Progressable progress) throws IOException {
LOG.trace("create() path:{}", f); LOG.trace("create() path:{}", f);
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.INVOCATION_CREATE, 1); storageStatistics.incrementCounter(Statistic.INVOCATION_CREATE, 1);
}
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
final String key = pathToKey(f); final String key = pathToKey(f);
final FileStatus status; final FileStatus status;
@ -240,8 +260,10 @@ public FSDataOutputStream createNonRecursive(Path path,
short replication, short replication,
long blockSize, long blockSize,
Progressable progress) throws IOException { Progressable progress) throws IOException {
if (storageStatistics != null) {
storageStatistics.incrementCounter( storageStatistics.incrementCounter(
Statistic.INVOCATION_CREATE_NON_RECURSIVE, 1); Statistic.INVOCATION_CREATE_NON_RECURSIVE, 1);
}
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
final Path parent = path.getParent(); final Path parent = path.getParent();
if (parent != null) { if (parent != null) {
@ -296,7 +318,9 @@ boolean processKey(String key) throws IOException {
*/ */
@Override @Override
public boolean rename(Path src, Path dst) throws IOException { public boolean rename(Path src, Path dst) throws IOException {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.INVOCATION_RENAME, 1); storageStatistics.incrementCounter(Statistic.INVOCATION_RENAME, 1);
}
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
if (src.equals(dst)) { if (src.equals(dst)) {
return true; return true;
@ -431,7 +455,9 @@ private boolean innerDelete(Path f, boolean recursive) throws IOException {
@Override @Override
public boolean delete(Path f, boolean recursive) throws IOException { public boolean delete(Path f, boolean recursive) throws IOException {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.INVOCATION_DELETE, 1); storageStatistics.incrementCounter(Statistic.INVOCATION_DELETE, 1);
}
statistics.incrementWriteOps(1); statistics.incrementWriteOps(1);
LOG.debug("Delete path {} - recursive {}", f, recursive); LOG.debug("Delete path {} - recursive {}", f, recursive);
FileStatus status; FileStatus status;
@ -623,7 +649,9 @@ FileStatus[] getStatuses() {
@Override @Override
public FileStatus[] listStatus(Path f) throws IOException { public FileStatus[] listStatus(Path f) throws IOException {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.INVOCATION_LIST_STATUS, 1); storageStatistics.incrementCounter(Statistic.INVOCATION_LIST_STATUS, 1);
}
statistics.incrementReadOps(1); statistics.incrementReadOps(1);
LOG.trace("listStatus() path:{}", f); LOG.trace("listStatus() path:{}", f);
ListStatusIterator iterator = new ListStatusIterator(f); ListStatusIterator iterator = new ListStatusIterator(f);
@ -710,7 +738,10 @@ public boolean mkdirs(Path f, FsPermission permission) throws IOException {
@Override @Override
public FileStatus getFileStatus(Path f) throws IOException { public FileStatus getFileStatus(Path f) throws IOException {
storageStatistics.incrementCounter(Statistic.INVOCATION_GET_FILE_STATUS, 1); if (storageStatistics != null) {
storageStatistics
.incrementCounter(Statistic.INVOCATION_GET_FILE_STATUS, 1);
}
statistics.incrementReadOps(1); statistics.incrementReadOps(1);
LOG.trace("getFileStatus() path:{}", f); LOG.trace("getFileStatus() path:{}", f);
Path qualifiedPath = f.makeQualified(uri, workingDir); Path qualifiedPath = f.makeQualified(uri, workingDir);