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

This commit is contained in:
Mukul Kumar Singh 2019-03-25 17:03:30 +05:30
parent d4e4a7d456
commit 67dd45fc25
22 changed files with 441 additions and 108 deletions

View File

@ -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.

View File

@ -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;
}

View File

@ -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.");
}
}

View File

@ -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());

View File

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

View File

@ -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() {

View File

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

View File

@ -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();
}
}
}

View File

@ -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;
}

View File

@ -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());
}
}

View File

@ -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;
}

View File

@ -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.

View File

@ -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;
}
}
}

View File

@ -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();

View File

@ -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.
*/

View File

@ -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;
}

View File

@ -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.
*/

View File

@ -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();
}
}

View File

@ -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;
}

View File

@ -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) {
public OzoneFileStatus getFileStatus(String pathKey) throws IOException {
try {
if (storageStatistics != null) {
storageStatistics.incrementCounter(Statistic.OBJECTS_LIST, 1);
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

View File

@ -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());
}
/**

View File

@ -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 {