Browse Source

HDDS-1033. Add FSStatistics for OzoneFileSystem.
Contributed by Mukul Kumar Singh.

Nanda kumar 6 years ago
parent
commit
394a9f7d5c

+ 4 - 3
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterFactory.java

@@ -40,7 +40,7 @@ public final class OzoneClientAdapterFactory {
 
   public static OzoneClientAdapter createAdapter(
       String volumeStr,
-      String bucketStr)
+      String bucketStr, OzoneFSStorageStatistics storageStatistics)
       throws IOException {
 
     ClassLoader currentClassLoader = OzoneFileSystem.class.getClassLoader();
@@ -70,10 +70,11 @@ public final class OzoneClientAdapterFactory {
 
       OzoneClientAdapter ozoneClientAdapter = (OzoneClientAdapter) classLoader
           .loadClass("org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl")
-          .getConstructor(String.class, String.class)
+          .getConstructor(String.class, String.class,
+              OzoneFSStorageStatistics.class)
           .newInstance(
               volumeStr,
-              bucketStr);
+              bucketStr, storageStatistics);
 
       Thread.currentThread().setContextClassLoader(contextClassLoader);
 

+ 15 - 4
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java

@@ -52,10 +52,11 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   private OzoneBucket bucket;
   private ReplicationType replicationType;
   private ReplicationFactor replicationFactor;
+  private OzoneFSStorageStatistics storageStatistics;
 
-  public OzoneClientAdapterImpl(String volumeStr, String bucketStr)
-      throws IOException {
-    this(createConf(), volumeStr, bucketStr);
+  public OzoneClientAdapterImpl(String volumeStr, String bucketStr,
+      OzoneFSStorageStatistics storageStatistics) throws IOException {
+    this(createConf(), volumeStr, bucketStr, storageStatistics);
   }
 
   private static OzoneConfiguration createConf() {
@@ -68,7 +69,8 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   }
 
   public OzoneClientAdapterImpl(OzoneConfiguration conf, String volumeStr,
-      String bucketStr) throws IOException {
+      String bucketStr, OzoneFSStorageStatistics storageStatistics)
+      throws IOException {
     ClassLoader contextClassLoader =
         Thread.currentThread().getContextClassLoader();
     Thread.currentThread().setContextClassLoader(null);
@@ -86,6 +88,7 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
       this.bucket = volume.getBucket(bucketStr);
       this.replicationType = ReplicationType.valueOf(replicationTypeConf);
       this.replicationFactor = ReplicationFactor.valueOf(replicationCountConf);
+      this.storageStatistics = storageStatistics;
     } finally {
       Thread.currentThread().setContextClassLoader(contextClassLoader);
     }
@@ -99,11 +102,13 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
 
   @Override
   public InputStream createInputStream(String key) throws IOException {
+    storageStatistics.incrementCounter(Statistic.OBJECTS_READ, 1);
     return bucket.readKey(key).getInputStream();
   }
 
   @Override
   public OzoneFSOutputStream createKey(String key) throws IOException {
+    storageStatistics.incrementCounter(Statistic.OBJECTS_CREATED, 1);
     OzoneOutputStream ozoneOutputStream =
         bucket.createKey(key, 0, replicationType, replicationFactor,
             new HashMap<>());
@@ -112,6 +117,7 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
 
   @Override
   public void renameKey(String key, String newKeyName) throws IOException {
+    storageStatistics.incrementCounter(Statistic.OBJECTS_RENAMED, 1);
     bucket.renameKey(key, newKeyName);
   }
 
@@ -124,6 +130,7 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   @Override
   public BasicKeyInfo getKeyInfo(String keyName) {
     try {
+      storageStatistics.incrementCounter(Statistic.OBJECTS_QUERY, 1);
       OzoneKey key = bucket.getKey(keyName);
       return new BasicKeyInfo(
           keyName,
@@ -160,6 +167,7 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   public boolean createDirectory(String keyName) {
     try {
       LOG.trace("creating dir for key:{}", keyName);
+      storageStatistics.incrementCounter(Statistic.OBJECTS_CREATED, 1);
       bucket.createKey(keyName, 0, replicationType, replicationFactor,
           new HashMap<>()).close();
       return true;
@@ -179,6 +187,7 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
   public boolean deleteObject(String keyName) {
     LOG.trace("issuing delete for key" + keyName);
     try {
+      storageStatistics.incrementCounter(Statistic.OBJECTS_DELETED, 1);
       bucket.deleteKey(keyName);
       return true;
     } catch (IOException ioe) {
@@ -194,11 +203,13 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
 
   @Override
   public boolean hasNextKey(String key) {
+    storageStatistics.incrementCounter(Statistic.OBJECTS_LIST, 1);
     return bucket.listKeys(key).hasNext();
   }
 
   @Override
   public Iterator<BasicKeyInfo> listKeys(String pathKey) {
+    storageStatistics.incrementCounter(Statistic.OBJECTS_LIST, 1);
     return new IteratorAdapter(bucket.listKeys(pathKey));
   }
 

+ 126 - 0
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSStorageStatistics.java

@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.ozone;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageStatistics;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.Map;
+import java.util.EnumMap;
+import java.util.Collections;
+import java.util.NoSuchElementException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Storage statistics for OzoneFileSystem.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class OzoneFSStorageStatistics extends StorageStatistics
+    implements Iterable<StorageStatistics.LongStatistic> {
+  private static final Logger LOG =
+      LoggerFactory.getLogger(OzoneFSStorageStatistics.class);
+
+  public static final String NAME = "OzoneFSStorageStatistics";
+  private final Map<Statistic, AtomicLong> opsCount =
+      new EnumMap<>(Statistic.class);
+
+  public OzoneFSStorageStatistics() {
+    super(NAME);
+    for (Statistic opType : Statistic.values()) {
+      opsCount.put(opType, new AtomicLong(0));
+    }
+  }
+
+  /**
+   * Increment a specific counter.
+   * @param op operation
+   * @param count increment value
+   * @return the new value
+   */
+  public long incrementCounter(Statistic op, long count) {
+    long updated = opsCount.get(op).addAndGet(count);
+    LOG.debug("{} += {}  ->  {}", op, count, updated);
+    return updated;
+  }
+
+  private class LongIterator implements Iterator<LongStatistic> {
+    private Iterator<Map.Entry<Statistic, AtomicLong>> iterator =
+        Collections.unmodifiableSet(opsCount.entrySet()).iterator();
+
+    @Override
+    public boolean hasNext() {
+      return iterator.hasNext();
+    }
+
+    @Override
+    public LongStatistic next() {
+      if (!iterator.hasNext()) {
+        throw new NoSuchElementException();
+      }
+      final Map.Entry<Statistic, AtomicLong> entry = iterator.next();
+      return new LongStatistic(entry.getKey().getSymbol(),
+          entry.getValue().get());
+    }
+
+    @Override
+    public void remove() {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  @Override
+  public String getScheme() {
+    return OzoneConsts.OZONE_URI_SCHEME;
+  }
+
+  @Override
+  public Iterator<LongStatistic> getLongStatistics() {
+    return new LongIterator();
+  }
+
+  @Override
+  public Iterator<LongStatistic> iterator() {
+    return getLongStatistics();
+  }
+
+  @Override
+  public Long getLong(String key) {
+    final Statistic type = Statistic.fromSymbol(key);
+    return type == null ? null : opsCount.get(type).get();
+  }
+
+  @Override
+  public boolean isTracked(String key) {
+    return Statistic.fromSymbol(key) != null;
+  }
+
+  @Override
+  public void reset() {
+    for (AtomicLong value : opsCount.values()) {
+      value.set(0);
+    }
+  }
+
+}

+ 35 - 4
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
+import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -84,6 +85,9 @@ public class OzoneFileSystem extends FileSystem {
 
   private OzoneClientAdapter adapter;
 
+
+  private OzoneFSStorageStatistics storageStatistics;
+
   private static final Pattern URL_SCHEMA_PATTERN =
       Pattern.compile("(.+)\\.([^\\.]+)");
 
@@ -121,9 +125,14 @@ public class OzoneFileSystem extends FileSystem {
       boolean isolatedClassloader =
           conf.getBoolean("ozone.fs.isolated-classloader", defaultValue);
 
+      storageStatistics = (OzoneFSStorageStatistics)
+          GlobalStorageStatistics.INSTANCE
+              .put(OzoneFSStorageStatistics.NAME,
+                  OzoneFSStorageStatistics::new);
       if (isolatedClassloader) {
         this.adapter =
-            OzoneClientAdapterFactory.createAdapter(volumeStr, bucketStr);
+            OzoneClientAdapterFactory.createAdapter(volumeStr, bucketStr,
+                storageStatistics);
       } else {
         OzoneConfiguration ozoneConfiguration;
         if (conf instanceof OzoneConfiguration) {
@@ -132,7 +141,7 @@ public class OzoneFileSystem extends FileSystem {
           ozoneConfiguration = new OzoneConfiguration(conf);
         }
         this.adapter = new OzoneClientAdapterImpl(ozoneConfiguration,
-            volumeStr, bucketStr);
+            volumeStr, bucketStr, storageStatistics);
       }
 
       try {
@@ -169,8 +178,18 @@ public class OzoneFileSystem extends FileSystem {
     return OZONE_URI_SCHEME;
   }
 
+  Statistics getFsStatistics() {
+    return statistics;
+  }
+
+  OzoneFSStorageStatistics getOzoneFSOpsCountStatistics() {
+    return storageStatistics;
+  }
+
   @Override
   public FSDataInputStream open(Path f, int bufferSize) throws IOException {
+    storageStatistics.incrementCounter(Statistic.INVOCATION_OPEN, 1);
+    statistics.incrementWriteOps(1);
     LOG.trace("open() path:{}", f);
     final FileStatus fileStatus = getFileStatus(f);
     final String key = pathToKey(f);
@@ -188,6 +207,8 @@ public class OzoneFileSystem extends FileSystem {
                                    short replication, long blockSize,
                                    Progressable progress) throws IOException {
     LOG.trace("create() path:{}", f);
+     storageStatistics.incrementCounter(Statistic.INVOCATION_CREATE, 1);
+    statistics.incrementWriteOps(1);
     final String key = pathToKey(f);
     final FileStatus status;
     try {
@@ -208,8 +229,7 @@ public class OzoneFileSystem extends FileSystem {
 
     // We pass null to FSDataOutputStream so it won't count writes that
     // are being buffered to a file
-    return new FSDataOutputStream(
-        adapter.createKey(key), null);
+    return new FSDataOutputStream(adapter.createKey(key), statistics);
   }
 
   @Override
@@ -220,6 +240,9 @@ public class OzoneFileSystem extends FileSystem {
       short replication,
       long blockSize,
       Progressable progress) throws IOException {
+    storageStatistics.incrementCounter(
+        Statistic.INVOCATION_CREATE_NON_RECURSIVE, 1);
+    statistics.incrementWriteOps(1);
     final Path parent = path.getParent();
     if (parent != null) {
       // expect this to raise an exception if there is no parent
@@ -273,6 +296,8 @@ public class OzoneFileSystem extends FileSystem {
    */
   @Override
   public boolean rename(Path src, Path dst) throws IOException {
+    storageStatistics.incrementCounter(Statistic.INVOCATION_RENAME, 1);
+    statistics.incrementWriteOps(1);
     if (src.equals(dst)) {
       return true;
     }
@@ -406,6 +431,8 @@ public class OzoneFileSystem extends FileSystem {
 
   @Override
   public boolean delete(Path f, boolean recursive) throws IOException {
+    storageStatistics.incrementCounter(Statistic.INVOCATION_DELETE, 1);
+    statistics.incrementWriteOps(1);
     LOG.debug("Delete path {} - recursive {}", f, recursive);
     FileStatus status;
     try {
@@ -596,6 +623,8 @@ public class OzoneFileSystem extends FileSystem {
 
   @Override
   public FileStatus[] listStatus(Path f) throws IOException {
+    storageStatistics.incrementCounter(Statistic.INVOCATION_LIST_STATUS, 1);
+    statistics.incrementReadOps(1);
     LOG.trace("listStatus() path:{}", f);
     ListStatusIterator iterator = new ListStatusIterator(f);
     iterator.iterate();
@@ -681,6 +710,8 @@ public class OzoneFileSystem extends FileSystem {
 
   @Override
   public FileStatus getFileStatus(Path f) throws IOException {
+    storageStatistics.incrementCounter(Statistic.INVOCATION_GET_FILE_STATUS, 1);
+    statistics.incrementReadOps(1);
     LOG.trace("getFileStatus() path:{}", f);
     Path qualifiedPath = f.makeQualified(uri, workingDir);
     String key = pathToKey(qualifiedPath);

+ 119 - 0
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/Statistic.java

@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.fs.ozone;
+
+import org.apache.hadoop.fs.StorageStatistics.CommonStatisticNames;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Statistic which are collected in OzoneFileSystem.
+ * These statistics are available at a low level in
+ * {@link OzoneFSStorageStatistics}
+ */
+public enum Statistic {
+  OBJECTS_RENAMED("objects_renamed",
+      "Total number of objects renamed within the object store."),
+  OBJECTS_CREATED("objects_created",
+      "Total number of objects created through the object store."),
+  OBJECTS_DELETED("objects_deleted",
+      "Total number of objects deleted from the object store."),
+  OBJECTS_READ("objects_read",
+      "Total number of objects read from the object store."),
+  OBJECTS_QUERY("objects_query",
+      "Total number of objects queried from the object store."),
+  OBJECTS_LIST("objects_list",
+      "Total number of object list query from the object store."),
+  INVOCATION_COPY_FROM_LOCAL_FILE(CommonStatisticNames.OP_COPY_FROM_LOCAL_FILE,
+      "Calls of copyFromLocalFile()"),
+  INVOCATION_CREATE(CommonStatisticNames.OP_CREATE,
+      "Calls of create()"),
+  INVOCATION_CREATE_NON_RECURSIVE(CommonStatisticNames.OP_CREATE_NON_RECURSIVE,
+      "Calls of createNonRecursive()"),
+  INVOCATION_DELETE(CommonStatisticNames.OP_DELETE,
+      "Calls of delete()"),
+  INVOCATION_EXISTS(CommonStatisticNames.OP_EXISTS,
+      "Calls of exists()"),
+  INVOCATION_GET_FILE_CHECKSUM(CommonStatisticNames.OP_GET_FILE_CHECKSUM,
+      "Calls of getFileChecksum()"),
+  INVOCATION_GET_FILE_STATUS(CommonStatisticNames.OP_GET_FILE_STATUS,
+      "Calls of getFileStatus()"),
+  INVOCATION_GLOB_STATUS(CommonStatisticNames.OP_GLOB_STATUS,
+      "Calls of globStatus()"),
+  INVOCATION_IS_DIRECTORY(CommonStatisticNames.OP_IS_DIRECTORY,
+      "Calls of isDirectory()"),
+  INVOCATION_IS_FILE(CommonStatisticNames.OP_IS_FILE,
+      "Calls of isFile()"),
+  INVOCATION_LIST_FILES(CommonStatisticNames.OP_LIST_FILES,
+      "Calls of listFiles()"),
+  INVOCATION_LIST_LOCATED_STATUS(CommonStatisticNames.OP_LIST_LOCATED_STATUS,
+      "Calls of listLocatedStatus()"),
+  INVOCATION_LIST_STATUS(CommonStatisticNames.OP_LIST_STATUS,
+      "Calls of listStatus()"),
+  INVOCATION_MKDIRS(CommonStatisticNames.OP_MKDIRS,
+      "Calls of mkdirs()"),
+  INVOCATION_OPEN(CommonStatisticNames.OP_OPEN,
+      "Calls of open()"),
+  INVOCATION_RENAME(CommonStatisticNames.OP_RENAME,
+      "Calls of rename()");
+
+  private static final Map<String, Statistic> SYMBOL_MAP =
+      new HashMap<>(Statistic.values().length);
+  static {
+    for (Statistic stat : values()) {
+      SYMBOL_MAP.put(stat.getSymbol(), stat);
+    }
+  }
+
+  Statistic(String symbol, String description) {
+    this.symbol = symbol;
+    this.description = description;
+  }
+
+  private final String symbol;
+  private final String description;
+
+  public String getSymbol() {
+    return symbol;
+  }
+
+  /**
+   * Get a statistic from a symbol.
+   * @param symbol statistic to look up
+   * @return the value or null.
+   */
+  public static Statistic fromSymbol(String symbol) {
+    return SYMBOL_MAP.get(symbol);
+  }
+
+  public String getDescription() {
+    return description;
+  }
+
+  /**
+   * The string value is simply the symbol.
+   * This makes this operation very low cost.
+   * @return the symbol of this statistic.
+   */
+  @Override
+  public String toString() {
+    return symbol;
+  }
+}

+ 27 - 6
hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileInterfaces.java

@@ -42,6 +42,8 @@ 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.apache.hadoop.fs.StorageStatistics;
+import org.apache.hadoop.fs.GlobalStorageStatistics;
 import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
 import org.apache.hadoop.ozone.web.handlers.BucketArgs;
 import org.apache.hadoop.ozone.web.handlers.UserArgs;
@@ -55,6 +57,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.hadoop.fs.ozone.Constants.OZONE_DEFAULT_USER;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.assertFalse;
 
 /**
  * Test OzoneFileSystem Interfaces.
@@ -88,12 +91,17 @@ public class TestOzoneFileInterfaces {
 
   private static FileSystem fs;
 
+  private static OzoneFileSystem o3fs;
+
   private static StorageHandler storageHandler;
 
+  private OzoneFSStorageStatistics statistics;
+
   public TestOzoneFileInterfaces(boolean setDefaultFs,
       boolean useAbsolutePath) {
     this.setDefaultFs = setDefaultFs;
     this.useAbsolutePath = useAbsolutePath;
+    GlobalStorageStatistics.INSTANCE.reset();
   }
 
   @Before
@@ -130,6 +138,8 @@ public class TestOzoneFileInterfaces {
     } else {
       fs = FileSystem.get(new URI(rootPath + "/test.txt"), conf);
     }
+    o3fs = (OzoneFileSystem) fs;
+    statistics = o3fs.getOzoneFSOpsCountStatistics();
   }
 
   @After
@@ -149,6 +159,7 @@ public class TestOzoneFileInterfaces {
               fs.getClass(),
           fs instanceof OzoneFileSystem);
       assertEquals(OzoneConsts.OZONE_URI_SCHEME, fs.getUri().getScheme());
+      assertEquals(OzoneConsts.OZONE_URI_SCHEME, statistics.getScheme());
     }
   }
 
@@ -163,13 +174,21 @@ public class TestOzoneFileInterfaces {
       stream.writeBytes(data);
     }
 
+    assertEquals(statistics.getLong(
+        StorageStatistics.CommonStatisticNames.OP_CREATE).longValue(), 1);
+    assertEquals(statistics.getLong("objects_created").longValue(), 1);
+
     FileStatus status = fs.getFileStatus(path);
+    assertEquals(statistics.getLong(
+        StorageStatistics.CommonStatisticNames.OP_GET_FILE_STATUS).longValue(),
+        2);
+    assertEquals(statistics.getLong("objects_query").longValue(), 1);
     // The timestamp of the newly created file should always be greater than
     // the time when the test was started
     assertTrue("Modification time has not been recorded: " + status,
         status.getModificationTime() > currentTime);
 
-    assertEquals(false, status.isDirectory());
+    assertFalse(status.isDirectory());
     assertEquals(FsPermission.getFileDefault(), status.getPermission());
     verifyOwnerGroup(status);
 
@@ -188,6 +207,9 @@ public class TestOzoneFileInterfaces {
       assertEquals(readBytes, buffer.length);
       assertEquals(buffer.length, inputStream.getPos());
     }
+    assertEquals(statistics.getLong(
+        StorageStatistics.CommonStatisticNames.OP_OPEN).longValue(), 1);
+    assertEquals(statistics.getLong("objects_read").longValue(), 1);
   }
 
   private void verifyOwnerGroup(FileStatus fileStatus) {
@@ -207,7 +229,7 @@ public class TestOzoneFileInterfaces {
     FileStatus status = fs.getFileStatus(path);
     assertTrue("The created path is not directory.", status.isDirectory());
 
-    assertEquals(true, status.isDirectory());
+    assertTrue(status.isDirectory());
     assertEquals(FsPermission.getDirDefault(), status.getPermission());
     verifyOwnerGroup(status);
 
@@ -224,15 +246,14 @@ public class TestOzoneFileInterfaces {
 
   @Test
   public void testPathToKey() throws Exception {
-    OzoneFileSystem ozoneFs = (OzoneFileSystem) TestOzoneFileInterfaces.fs;
 
-    assertEquals("a/b/1", ozoneFs.pathToKey(new Path("/a/b/1")));
+    assertEquals("a/b/1", o3fs.pathToKey(new Path("/a/b/1")));
 
     assertEquals("user/" + getCurrentUser() + "/key1/key2",
-        ozoneFs.pathToKey(new Path("key1/key2")));
+        o3fs.pathToKey(new Path("key1/key2")));
 
     assertEquals("key1/key2",
-        ozoneFs.pathToKey(new Path("o3fs://test1/key1/key2")));
+        o3fs.pathToKey(new Path("o3fs://test1/key1/key2")));
   }
 
   private String getCurrentUser() {