Browse Source

HDDS-1185. Optimize GetFileStatus in OzoneFileSystem by reducing the number of rpc call to OM. Contributed by Mukul Kumar Singh.

Mukul Kumar Singh 6 years ago
parent
commit
67dd45fc25
22 changed files with 441 additions and 108 deletions
  1. 4 2
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
  2. 12 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java
  3. 9 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java
  4. 7 0
      hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
  5. 1 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
  6. 4 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java
  7. 3 1
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
  8. 97 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFileStatus.java
  9. 12 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java
  10. 32 0
      hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
  11. 17 0
      hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
  12. 2 1
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java
  13. 58 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
  14. 25 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java
  15. 23 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java
  16. 31 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/fs/OzoneManagerFS.java
  17. 21 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/fs/package-info.java
  18. 17 0
      hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
  19. 4 6
      hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java
  20. 15 34
      hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapterImpl.java
  21. 4 60
      hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneFileSystem.java
  22. 43 3
      hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileInterfaces.java

+ 4 - 2
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
 import org.apache.hadoop.ozone.OzoneAcl;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.helpers.WithMetadata;
 
 import java.io.IOException;
@@ -464,8 +465,9 @@ public class OzoneBucket extends WithMetadata {
               partNumberMarker, maxParts);
   }
 
-
-
+  public OzoneFileStatus getFileStatus(String keyName) throws IOException {
+    return proxy.getOzoneFileStatus(volumeName, name, keyName);
+  }
 
   /**
    * An Iterator to iterate over {@link OzoneKey} list.

+ 12 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/protocol/ClientProtocol.java

@@ -39,6 +39,7 @@ import java.net.URI;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.security.KerberosInfo;
@@ -533,4 +534,15 @@ public interface ClientProtocol {
    * @return Canonical Service Name of ozone delegation token.
    */
   String getCanonicalServiceName();
+
+  /**
+   * Get the Ozone File Status for a particular Ozone key.
+   * @param volumeName volume name.
+   * @param bucketName bucket name.
+   * @param keyName key name.
+   * @return OzoneFileStatus for the key.
+   * @throws IOException
+   */
+  OzoneFileStatus getOzoneFileStatus(String volumeName,
+      String bucketName, String keyName) throws IOException;
 }

+ 9 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rest/RestClient.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.ozone.client.*;
 import org.apache.hadoop.hdds.client.OzoneQuota;
 import org.apache.hadoop.hdds.client.ReplicationFactor;
 import org.apache.hadoop.hdds.client.ReplicationType;
+import org.apache.hadoop.ozone.client.VolumeArgs;
 import org.apache.hadoop.ozone.client.io.OzoneInputStream;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
@@ -50,6 +51,7 @@ import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.ozone.web.response.ListBuckets;
@@ -1082,4 +1084,11 @@ public class RestClient implements ClientProtocol {
     throw new UnsupportedOperationException("Ozone REST protocol does not " +
         "support this operation.");
   }
+
+  @Override
+  public OzoneFileStatus getOzoneFileStatus(String volumeName,
+      String bucketName, String keyName) throws IOException {
+    throw new UnsupportedOperationException("Ozone REST protocol does not " +
+        "support this operation.");
+  }
 }

+ 7 - 0
hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java

@@ -66,6 +66,7 @@ import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
 import org.apache.hadoop.ozone.om.protocolPB
     .OzoneManagerProtocolClientSideTranslatorPB;
@@ -970,6 +971,12 @@ public class RpcClient implements ClientProtocol, KeyProviderTokenIssuer {
 
   }
 
+  @Override
+  public OzoneFileStatus getOzoneFileStatus(String volumeName,
+      String bucketName, String keyName) throws IOException {
+    return ozoneManagerClient.getFileStatus(volumeName, bucketName, keyName);
+  }
+
   @Override
   public KeyProvider getKeyProvider() throws IOException {
     return OzoneKMSUtil.getKeyProvider(conf, getKeyProviderUri());

+ 1 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java

@@ -187,6 +187,7 @@ public final class OmUtils {
     case ListS3Buckets:
     case ServiceList:
     case ListMultiPartUploadParts:
+    case GetFileStatus:
       return true;
     case CreateVolume:
     case SetVolumeProperty:

+ 4 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/audit/OMAction.java

@@ -51,7 +51,10 @@ public enum OMAction implements AuditAction {
   INITIATE_MULTIPART_UPLOAD,
   COMMIT_MULTIPART_UPLOAD_PARTKEY,
   COMPLETE_MULTIPART_UPLOAD,
-  LIST_MULTIPART_UPLOAD_PARTS;
+  LIST_MULTIPART_UPLOAD_PARTS,
+
+  //FS Actions
+  GET_FILE_STATUS;
 
   @Override
   public String getAction() {

+ 3 - 1
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java

@@ -185,6 +185,8 @@ public class OMException extends IOException {
 
     INVALID_KMS_PROVIDER,
 
-    TOKEN_CREATION_ERROR
+    TOKEN_CREATION_ERROR,
+
+    FILE_NOT_FOUND
   }
 }

+ 97 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/helpers/OzoneFileStatus.java

@@ -0,0 +1,97 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.om.helpers;
+
+import org.apache.hadoop.fs.FSProtos.FileStatusProto;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.fs.protocolPB.PBHelper;
+
+import java.io.IOException;
+import java.net.URI;
+
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
+
+/**
+ * File Status of the Ozone Key.
+ */
+public class OzoneFileStatus extends FileStatus {
+  public OzoneFileStatus(OmKeyInfo key, long blockSize, boolean isDirectory) {
+    super(key.getDataSize(), isDirectory, key.getFactor().getNumber(),
+        blockSize, key.getModificationTime(), getPath(key.getKeyName()));
+  }
+
+  public OzoneFileStatus(FileStatus status) throws IOException {
+    super(status);
+  }
+
+  // Use this constructor only for directories
+  public OzoneFileStatus(int replication, long blockSize,
+                         String keyName) {
+    super(0, true, replication, blockSize, 0,
+        getPath(keyName));
+  }
+
+  public FileStatusProto getProtobuf() throws IOException {
+    return PBHelper.convert(this);
+  }
+
+  public static OzoneFileStatus getFromProtobuf(FileStatusProto response)
+      throws IOException {
+    return new OzoneFileStatus(PBHelper.convert(response));
+  }
+
+  public static Path getPath(String keyName) {
+    return new Path(OZONE_URI_DELIMITER + keyName);
+  }
+
+  public FileStatus makeQualified(URI defaultUri, Path parent,
+                                  String owner, String group) {
+    // fully-qualify path
+    setPath(parent.makeQualified(defaultUri, null));
+    setGroup(group);
+    setOwner(owner);
+    if (isDirectory()) {
+      setPermission(FsPermission.getDirDefault());
+    } else {
+      setPermission(FsPermission.getFileDefault());
+    }
+    return this; // API compatibility
+  }
+
+  /** Get the modification time of the file/directory.
+   *
+   * o3fs uses objects as "fake" directories, which are not updated to
+   * reflect the accurate modification time. We choose to report the
+   * current time because some parts of the ecosystem (e.g. the
+   * HistoryServer) use modification time to ignore "old" directories.
+   *
+   * @return for files the modification time in milliseconds since January 1,
+   *         1970 UTC or for directories the current time.
+   */
+  @Override
+  public long getModificationTime(){
+    if (isDirectory()) {
+      return System.currentTimeMillis();
+    } else {
+      return super.getModificationTime();
+    }
+  }
+}

+ 12 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocol/OzoneManagerProtocol.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo;
 
 import java.io.Closeable;
@@ -397,5 +398,16 @@ public interface OzoneManagerProtocol
    * @return OMFailoverProxyProvider
    */
   OMFailoverProxyProvider getOMFailoverProxyProvider();
+
+  /**
+   * Get File Status for an Ozone key.
+   * @param volumeName volume name.
+   * @param bucketName bucket name.
+   * @param keyName key name.
+   * @return OzoneFileStatus for the key.
+   * @throws IOException
+   */
+  OzoneFileStatus getFileStatus(String volumeName, String bucketName,
+                                String keyName) throws IOException;
 }
 

+ 32 - 0
hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java

@@ -53,7 +53,10 @@ import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
 import org.apache.hadoop.ozone.om.helpers.S3SecretValue;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AllocateBlockRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AllocateBlockResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.BucketArgs;
@@ -1219,4 +1222,33 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
     }
   }
 
+  /**
+   * Get File Status for an Ozone key.
+   * @param volumeName volume name.
+   * @param bucketName bucket name.
+   * @param keyName key name.
+   * @return OzoneFileStatus for the key.
+   * @throws IOException
+   */
+  public OzoneFileStatus getFileStatus(String volumeName, String bucketName,
+                                String keyName) throws IOException {
+    GetFileStatusRequest req = GetFileStatusRequest
+        .newBuilder()
+        .setVolumeName(volumeName)
+        .setBucketName(bucketName)
+        .setKeyName(keyName)
+        .build();
+
+    OMRequest omRequest = createOMRequest(Type.GetFileStatus)
+        .setGetFileStatusRequest(req)
+        .build();
+
+    final GetFileStatusResponse resp;
+    try {
+      resp = handleError(submitRequest(omRequest)).getGetFileStatusResponse();
+    } catch (IOException e) {
+      throw e;
+    }
+    return OzoneFileStatus.getFromProtobuf(resp.getStatus());
+  }
 }

+ 17 - 0
hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto

@@ -37,6 +37,7 @@ This is similar to Namenode for Ozone.
 
 import "hdds.proto";
 import "Security.proto";
+import "FSProtos.proto";
 
 enum Type {
   CreateVolume = 11;
@@ -76,6 +77,8 @@ enum Type {
   GetDelegationToken = 61;
   RenewDelegationToken = 62;
   CancelDelegationToken = 63;
+
+  GetFileStatus = 70;
 }
 
 message OMRequest {
@@ -125,6 +128,7 @@ message OMRequest {
   optional hadoop.common.RenewDelegationTokenRequestProto renewDelegationTokenRequest= 62;
   optional hadoop.common.CancelDelegationTokenRequestProto cancelDelegationTokenRequest = 63;
 
+  optional GetFileStatusRequest             getFileStatusRequest           = 70;
 }
 
 message OMResponse {
@@ -179,6 +183,8 @@ message OMResponse {
   optional GetDelegationTokenResponseProto getDelegationTokenResponse = 61;
   optional RenewDelegationTokenResponseProto renewDelegationTokenResponse = 62;
   optional CancelDelegationTokenResponseProto cancelDelegationTokenResponse = 63;
+
+  optional GetFileStatusResponse              getFileStatusResponse        = 70;
 }
 
 enum Status {
@@ -230,6 +236,7 @@ enum Status {
     INVALID_KMS_PROVIDER = 42;
     TOKEN_CREATION_ERROR = 43;
 
+    FILE_NOT_FOUND = 44;
 }
 
 
@@ -525,6 +532,16 @@ message KeyInfo {
     optional FileEncryptionInfoProto fileEncryptionInfo = 12;
 }
 
+message GetFileStatusRequest {
+    required string volumeName = 1;
+    required string bucketName = 2;
+    required string keyName = 3;
+}
+
+message GetFileStatusResponse {
+    required hadoop.fs.FileStatusProto status = 1;
+}
+
 message CreateKeyRequest {
     required KeyArgs keyArgs = 1;
 }

+ 2 - 1
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManager.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList;
 import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
 import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.fs.OzoneManagerFS;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
 import org.apache.hadoop.utils.BackgroundService;
 
@@ -37,7 +38,7 @@ import java.util.List;
 /**
  * Handles key level commands.
  */
-public interface KeyManager {
+public interface KeyManager extends OzoneManagerFS {
 
   /**
    * Start key manager.

+ 58 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java

@@ -31,6 +31,7 @@ import java.security.GeneralSecurityException;
 import java.security.PrivilegedExceptionAction;
 
 import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.conf.StorageUnit;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
 import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
@@ -49,6 +50,7 @@ import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
 import org.apache.hadoop.ozone.OzoneConsts;
 import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.hdds.protocol.proto.HddsProtos.BlockTokenSecretProto.AccessModeProto;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
 import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
 import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
@@ -92,6 +94,7 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_BL
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
 import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
 import static org.apache.hadoop.ozone.OzoneConsts.OM_MULTIPART_MIN_SIZE;
+import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
 import static org.apache.hadoop.util.Time.monotonicNow;
 
 import org.slf4j.Logger;
@@ -1276,4 +1279,59 @@ public class KeyManagerImpl implements KeyManager {
     }
   }
 
+  public OzoneFileStatus getFileStatus(String volumeName, String bucketName,
+                                       String keyName) throws IOException {
+    Preconditions.checkNotNull(volumeName);
+    Preconditions.checkNotNull(bucketName);
+    Preconditions.checkNotNull(keyName);
+
+    metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
+    try {
+      // Check if this is the root of the filesystem.
+      if (keyName.length() == 0) {
+        validateBucket(volumeName, bucketName);
+        return new OzoneFileStatus(3, scmBlockSize, keyName);
+      }
+
+      //Check if the key is a file.
+      String fileKeyBytes = metadataManager.getOzoneKey(
+          volumeName, bucketName, keyName);
+      OmKeyInfo fileKeyInfo = metadataManager.getKeyTable().get(fileKeyBytes);
+      if (fileKeyInfo != null) {
+        // this is a file
+        return new OzoneFileStatus(fileKeyInfo, scmBlockSize, false);
+      }
+
+      String dirKey = addTrailingSlashIfNeeded(keyName);
+      String dirKeyBytes = metadataManager.getOzoneKey(
+          volumeName, bucketName, dirKey);
+      OmKeyInfo dirKeyInfo = metadataManager.getKeyTable().get(dirKeyBytes);
+      if (dirKeyInfo != null) {
+        return new OzoneFileStatus(dirKeyInfo, scmBlockSize, true);
+      }
+
+      List<OmKeyInfo> keys = metadataManager.listKeys(volumeName, bucketName,
+          null, dirKey, 1);
+      if (keys.iterator().hasNext()) {
+        return new OzoneFileStatus(3, scmBlockSize, keyName);
+      }
+
+      LOG.debug("Unable to get file status for the key: volume:" + volumeName +
+          " bucket:" + bucketName + " key:" + keyName + " with error no " +
+          "such file exists:");
+      throw new OMException("Unable to get file status: volume: " +
+          volumeName + "bucket: " + bucketName + "key: " + keyName,
+          ResultCodes.FILE_NOT_FOUND);
+    } finally {
+      metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
+    }
+  }
+
+  private String addTrailingSlashIfNeeded(String key) {
+    if (StringUtils.isNotEmpty(key) && !key.endsWith(OZONE_URI_DELIMITER)) {
+      return key + OZONE_URI_DELIMITER;
+    } else {
+      return key;
+    }
+  }
 }

+ 25 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMMetrics.java

@@ -40,6 +40,7 @@ public class OMMetrics {
   private @Metric MutableCounterLong numVolumeOps;
   private @Metric MutableCounterLong numBucketOps;
   private @Metric MutableCounterLong numKeyOps;
+  private @Metric MutableCounterLong numFSOps;
 
   // OM op metrics
   private @Metric MutableCounterLong numVolumeCreates;
@@ -66,6 +67,7 @@ public class OMMetrics {
   private @Metric MutableCounterLong numInitiateMultipartUploads;
   private @Metric MutableCounterLong numCompleteMultipartUploads;
 
+  private @Metric MutableCounterLong numGetFileStatus;
 
   // Failure Metrics
   private @Metric MutableCounterLong numVolumeCreateFails;
@@ -98,6 +100,8 @@ public class OMMetrics {
   private @Metric MutableCounterLong numListMultipartUploadParts;
   private @Metric MutableCounterLong numListMultipartUploadPartFails;
 
+  private @Metric MutableCounterLong numGetFileStatusFails;
+
   // Metrics for total number of volumes, buckets and keys
 
   private @Metric MutableCounterLong numVolumes;
@@ -283,6 +287,16 @@ public class OMMetrics {
     numListMultipartUploadParts.incr();
   }
 
+  public void incNumGetFileStatus() {
+    numKeyOps.incr();
+    numFSOps.incr();
+    numGetFileStatus.incr();
+  }
+
+  public void incNumGetFileStatusFails() {
+    numGetFileStatusFails.incr();
+  }
+
   public void incNumListMultipartUploadPartFails() {
     numListMultipartUploadPartFails.incr();
   }
@@ -577,6 +591,17 @@ public class OMMetrics {
     return numKeyListFails.value();
   }
 
+
+  @VisibleForTesting
+  public long getNumFSOps() {
+    return numFSOps.value();
+  }
+
+  @VisibleForTesting
+  public long getNumGetFileStatus() {
+    return numGetFileStatus.value();
+  }
+
   @VisibleForTesting
   public long getNumVolumeListFails() {
     return numVolumeListFails.value();

+ 23 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java

@@ -103,6 +103,7 @@ import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
 import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
 import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolPB;
 import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisClient;
 import org.apache.hadoop.ozone.om.ratis.OzoneManagerRatisServer;
@@ -2591,6 +2592,28 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
     }
   }
 
+  @Override
+  public OzoneFileStatus getFileStatus(String volumeName, String bucketName,
+                                       String keyName) throws IOException {
+    Map<String, String> auditMap = new HashMap<>();
+    auditMap.put(OzoneConsts.VOLUME, volumeName);
+    auditMap.put(OzoneConsts.BUCKET, bucketName);
+    auditMap.put(OzoneConsts.KEY, keyName);
+    metrics.incNumGetFileStatus();
+    try {
+      OzoneFileStatus ozoneFileStatus =
+          keyManager.getFileStatus(volumeName, bucketName, keyName);
+      AUDIT.logWriteSuccess(buildAuditMessageForSuccess(OMAction
+          .GET_FILE_STATUS, auditMap));
+      return ozoneFileStatus;
+    } catch (IOException ex) {
+      metrics.incNumGetFileStatusFails();
+      AUDIT.logWriteFailure(buildAuditMessageForFailure(OMAction
+          .GET_FILE_STATUS, auditMap, ex));
+      throw ex;
+    }
+  }
+
   /**
    * Startup options.
    */

+ 31 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/fs/OzoneManagerFS.java

@@ -0,0 +1,31 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.om.fs;
+
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+
+import java.io.IOException;
+
+/**
+ * Ozone Manager FileSystem interface.
+ */
+public interface OzoneManagerFS {
+  OzoneFileStatus getFileStatus(String volumeName, String bucketName,
+                                String keyName) throws IOException;
+}

+ 21 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/fs/package-info.java

@@ -0,0 +1,21 @@
+/**
+ * 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.ozone.om.fs;
+/*
+ This package contains the Ozone Manager FileSystem interface classes.
+ */

+ 17 - 0
hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java

@@ -43,6 +43,8 @@ import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
 import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
 import org.apache.hadoop.ozone.om.protocol.OzoneManagerServerProtocol;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetFileStatusResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AllocateBlockRequest;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.AllocateBlockResponse;
 import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CancelDelegationTokenResponseProto;
@@ -310,6 +312,11 @@ public class OzoneManagerRequestHandler implements RequestHandler {
             .getGetS3SecretRequest());
         responseBuilder.setGetS3SecretResponse(getS3SecretResp);
         break;
+      case GetFileStatus:
+        GetFileStatusResponse getFileStatusResponse =
+            getOzoneFileStatus(request.getGetFileStatusRequest());
+        responseBuilder.setGetFileStatusResponse(getFileStatusResponse);
+        break;
       default:
         responseBuilder.setSuccess(false);
         responseBuilder.setMessage("Unrecognized Command Type: " + cmdType);
@@ -891,4 +898,14 @@ public class OzoneManagerRequestHandler implements RequestHandler {
 
     return rb.build();
   }
+
+  private GetFileStatusResponse getOzoneFileStatus(
+      GetFileStatusRequest request) throws IOException {
+    GetFileStatusResponse.Builder rb = GetFileStatusResponse.newBuilder();
+
+    rb.setStatus(impl.getFileStatus(request.getVolumeName(),
+        request.getBucketName(), request.getKeyName()).getProtobuf());
+
+    return rb.build();
+  }
 }

+ 4 - 6
hadoop-ozone/ozonefs/src/main/java/org/apache/hadoop/fs/ozone/OzoneClientAdapter.java

@@ -21,6 +21,8 @@ import org.apache.hadoop.crypto.key.KeyProvider;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
 import org.apache.hadoop.security.token.Token;
 
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
@@ -43,18 +45,12 @@ public interface OzoneClientAdapter {
 
   void renameKey(String key, String newKeyName) throws IOException;
 
-  BasicKeyInfo getKeyInfo(String keyName);
-
   boolean isDirectory(BasicKeyInfo key);
 
   boolean createDirectory(String keyName);
 
   boolean deleteObject(String keyName);
 
-  long getCreationTime();
-
-  boolean hasNextKey(String key);
-
   Iterator<BasicKeyInfo> listKeys(String pathKey);
 
   Token<OzoneTokenIdentifier> getDelegationToken(String renewer)
@@ -65,4 +61,6 @@ public interface OzoneClientAdapter {
   URI getKeyProviderUri() throws IOException;
 
   String getCanonicalServiceName();
+
+  OzoneFileStatus getFileStatus(String pathKey) throws IOException;
 }

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

@@ -19,6 +19,7 @@ package org.apache.hadoop.fs.ozone;
 
 import static org.apache.hadoop.ozone.OzoneConsts.OZONE_URI_DELIMITER;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.net.URI;
@@ -43,6 +44,8 @@ import org.apache.hadoop.ozone.client.OzoneKey;
 import org.apache.hadoop.ozone.client.OzoneVolume;
 import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.security.token.TokenRenewer;
 import org.slf4j.Logger;
@@ -186,30 +189,6 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
     bucket.renameKey(key, newKeyName);
   }
 
-  /**
-   * Helper method to fetch the key metadata info.
-   *
-   * @param keyName key whose metadata information needs to be fetched
-   * @return metadata info of the key
-   */
-  @Override
-  public BasicKeyInfo getKeyInfo(String keyName) {
-    try {
-      if (storageStatistics != null) {
-        storageStatistics.incrementCounter(Statistic.OBJECTS_QUERY, 1);
-      }
-      OzoneKey key = bucket.getKey(keyName);
-      return new BasicKeyInfo(
-          keyName,
-          key.getModificationTime(),
-          key.getDataSize()
-      );
-    } catch (IOException e) {
-      LOG.trace("Key:{} does not exist", keyName);
-      return null;
-    }
-  }
-
   /**
    * Helper method to check if an Ozone key is representing a directory.
    *
@@ -267,17 +246,19 @@ public class OzoneClientAdapterImpl implements OzoneClientAdapter {
     }
   }
 
-  @Override
-  public long getCreationTime() {
-    return bucket.getCreationTime();
-  }
-
-  @Override
-  public boolean hasNextKey(String key) {
-    if (storageStatistics != null) {
-      storageStatistics.incrementCounter(Statistic.OBJECTS_LIST, 1);
+  public OzoneFileStatus getFileStatus(String pathKey) throws IOException {
+    try {
+      if (storageStatistics != null) {
+        storageStatistics.incrementCounter(Statistic.OBJECTS_QUERY, 1);
+      }
+      return bucket.getFileStatus(pathKey);
+    } catch (OMException e) {
+      if (e.getResult() == OMException.ResultCodes.FILE_NOT_FOUND) {
+        throw new
+            FileNotFoundException(pathKey + ": No such file or directory!");
+      }
+      throw e;
     }
-    return bucket.listKeys(key).hasNext();
   }
 
   @Override

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

@@ -648,10 +648,10 @@ public class OzoneFileSystem extends FileSystem
      * @param dirPath path to the dir
      * @throws FileNotFoundException
      */
-    void addSubDirStatus(Path dirPath) throws FileNotFoundException {
+    void addSubDirStatus(Path dirPath) throws IOException {
       // Check if subdir path is already included in statuses.
       if (!subDirStatuses.containsKey(dirPath)) {
-        subDirStatuses.put(dirPath, innerGetFileStatusForDir(dirPath));
+        subDirStatuses.put(dirPath, getFileStatus(dirPath));
       }
     }
 
@@ -803,64 +803,8 @@ public class OzoneFileSystem extends FileSystem
     Path qualifiedPath = f.makeQualified(uri, workingDir);
     String key = pathToKey(qualifiedPath);
 
-    if (key.length() == 0) {
-      return new FileStatus(0, true, 1, 0,
-          adapter.getCreationTime(), qualifiedPath);
-    }
-
-    // Check if the key exists
-    BasicKeyInfo ozoneKey = adapter.getKeyInfo(key);
-    if (ozoneKey != null) {
-      LOG.debug("Found exact file for path {}: normal file", f);
-      return new FileStatus(ozoneKey.getDataSize(), false, 1,
-          getDefaultBlockSize(f), ozoneKey.getModificationTime(), 0,
-          FsPermission.getFileDefault(), getUsername(), getUsername(),
-          qualifiedPath);
-    }
-
-    return innerGetFileStatusForDir(f);
-  }
-
-  /**
-   * Get the FileStatus for input directory path.
-   * They key corresponding to input path is appended with a trailing slash
-   * to return only the corresponding directory key in the bucket.
-   *
-   * @param f directory path
-   * @return FileStatus for the input directory path
-   * @throws FileNotFoundException
-   */
-  public FileStatus innerGetFileStatusForDir(Path f)
-      throws FileNotFoundException {
-    Path qualifiedPath = f.makeQualified(uri, workingDir);
-    String key = pathToKey(qualifiedPath);
-    key = addTrailingSlashIfNeeded(key);
-
-    BasicKeyInfo ozoneKey = adapter.getKeyInfo(key);
-    if (ozoneKey != null) {
-      if (adapter.isDirectory(ozoneKey)) {
-        // Key is a directory
-        LOG.debug("Found file (with /) for path {}: fake directory", f);
-      } else {
-        // Key is a file with trailing slash
-        LOG.warn("Found file (with /) for path {}: real file? should not " +
-            "happen", f, key);
-      }
-      return new FileStatus(0, true, 1, 0,
-          ozoneKey.getModificationTime(), 0,
-          FsPermission.getDirDefault(), getUsername(), getUsername(),
-          qualifiedPath);
-    }
-
-    // File or directory corresponding to input path does not exist.
-    // Check if there exists a key prefixed with this key.
-    boolean hasChildren = adapter.hasNextKey(key);
-    if (hasChildren) {
-      return new FileStatus(0, true, 1, 0, 0, 0, FsPermission.getDirDefault(),
-          getUsername(), getUsername(), qualifiedPath);
-    }
-
-    throw new FileNotFoundException(f + ": No such file or directory!");
+    return adapter.getFileStatus(key)
+        .makeQualified(uri, qualifiedPath, getUsername(), getUsername());
   }
 
   /**

+ 43 - 3
hadoop-ozone/ozonefs/src/test/java/org/apache/hadoop/fs/ozone/TestOzoneFileInterfaces.java

@@ -26,6 +26,8 @@ import java.util.Collection;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.ozone.MiniOzoneCluster;
 import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -93,6 +95,10 @@ public class TestOzoneFileInterfaces {
 
   private static OzoneFileSystem o3fs;
 
+  private static String volumeName;
+
+  private static String bucketName;
+
   private static StorageHandler storageHandler;
 
   private OzoneFSStorageStatistics statistics;
@@ -117,8 +123,8 @@ public class TestOzoneFileInterfaces {
     // create a volume and a bucket to be used by OzoneFileSystem
     userName = "user" + RandomStringUtils.randomNumeric(5);
     String adminName = "admin" + RandomStringUtils.randomNumeric(5);
-    String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
-    String bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
+    volumeName = "volume" + RandomStringUtils.randomNumeric(5);
+    bucketName = "bucket" + RandomStringUtils.randomNumeric(5);
     UserArgs userArgs = new UserArgs(null, OzoneUtils.getRequestID(),
         null, null, null, null);
     VolumeArgs volumeArgs = new VolumeArgs(volumeName, userArgs);
@@ -182,7 +188,7 @@ public class TestOzoneFileInterfaces {
     assertEquals(statistics.getLong(
         StorageStatistics.CommonStatisticNames.OP_GET_FILE_STATUS).longValue(),
         2);
-    assertEquals(statistics.getLong("objects_query").longValue(), 1);
+    assertEquals(statistics.getLong("objects_query").longValue(), 2);
     // 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,
@@ -244,6 +250,40 @@ public class TestOzoneFileInterfaces {
     assertEquals(0, status.getLen());
   }
 
+  @Test
+  public void testOzoneManagerFileSystemInterface() throws IOException {
+    String dirPath = RandomStringUtils.randomAlphanumeric(5);
+
+    Path path = createPath("/" + dirPath);
+    assertTrue("Makedirs returned with false for the path " + path,
+        fs.mkdirs(path));
+
+    long numFileStatus =
+        cluster.getOzoneManager().getMetrics().getNumGetFileStatus();
+    FileStatus status = fs.getFileStatus(path);
+
+    Assert.assertEquals(numFileStatus + 1,
+        cluster.getOzoneManager().getMetrics().getNumGetFileStatus());
+    assertTrue(status.isDirectory());
+    assertEquals(FsPermission.getDirDefault(), status.getPermission());
+    verifyOwnerGroup(status);
+
+    long currentTime = System.currentTimeMillis();
+    OzoneFileStatus omStatus =
+        cluster.getOzoneManager().getFileStatus(volumeName,
+        bucketName, o3fs.pathToKey(path));
+    //Another get file status here, incremented the counter.
+    Assert.assertEquals(numFileStatus + 2,
+        cluster.getOzoneManager().getMetrics().getNumGetFileStatus());
+
+    assertTrue("The created path is not directory.", omStatus.isDirectory());
+
+    // For directories, the time returned is the current time.
+    assertEquals(0, omStatus.getLen());
+    assertTrue(omStatus.getModificationTime() >= currentTime);
+    assertEquals(omStatus.getPath().getName(), o3fs.pathToKey(path));
+  }
+
   @Test
   public void testPathToKey() throws Exception {