Преглед изворни кода

HDDS-1460: Add the optmizations of HDDS-1300 to BasicOzoneFileSystem (#765)

Lokesh Jain пре 6 година
родитељ
комит
556eafd01a

+ 1 - 33
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/BasicOzoneFileSystem.java

@@ -203,14 +203,8 @@ public class BasicOzoneFileSystem extends FileSystem {
     incrementCounter(Statistic.INVOCATION_OPEN);
     statistics.incrementWriteOps(1);
     LOG.trace("open() path:{}", f);
-    final FileStatus fileStatus = getFileStatus(f);
     final String key = pathToKey(f);
-    if (fileStatus.isDirectory()) {
-      throw new FileNotFoundException("Can't open directory " + f + " to read");
-    }
-
-    return new FSDataInputStream(
-        new OzoneFSInputStream(adapter.readFile(key)));
+    return new FSDataInputStream(new OzoneFSInputStream(adapter.readFile(key)));
   }
 
   protected void incrementCounter(Statistic statistic) {
@@ -226,25 +220,6 @@ public class BasicOzoneFileSystem extends FileSystem {
     incrementCounter(Statistic.INVOCATION_CREATE);
     statistics.incrementWriteOps(1);
     final String key = pathToKey(f);
-    final FileStatus status;
-    try {
-      status = getFileStatus(f);
-      if (status.isDirectory()) {
-        throw new FileAlreadyExistsException(f + " is a directory");
-      } else {
-        if (!overwrite) {
-          // path references a file and overwrite is disabled
-          throw new FileAlreadyExistsException(f + " already exists");
-        }
-        LOG.trace("Overwriting file {}", f);
-        adapter.deleteObject(key);
-      }
-    } catch (FileNotFoundException ignored) {
-      // this means the file is not found
-    }
-
-    // We pass null to FSDataOutputStream so it won't count writes that
-    // are being buffered to a file
     return createOutputStream(key, overwrite, true);
   }
 
@@ -259,13 +234,6 @@ public class BasicOzoneFileSystem extends FileSystem {
     incrementCounter(Statistic.INVOCATION_CREATE_NON_RECURSIVE);
     statistics.incrementWriteOps(1);
     final String key = pathToKey(path);
-    final Path parent = path.getParent();
-    if (parent != null) {
-      // expect this to raise an exception if there is no parent
-      if (!getFileStatus(parent).isDirectory()) {
-        throw new FileAlreadyExistsException("Not a directory: " + parent);
-      }
-    }
     return createOutputStream(key, flags.contains(CreateFlag.OVERWRITE), false);
   }