diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java index 493315e0add4..3a63a5934692 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneClient.java @@ -90,9 +90,10 @@ public OzoneClient(ConfigurationSource conf, ClientProtocol proxy) { } @VisibleForTesting - protected OzoneClient(ObjectStore objectStore) { + protected OzoneClient(ObjectStore objectStore, + ClientProtocol clientProtocol) { this.objectStore = objectStore; - this.proxy = null; + this.proxy = clientProtocol; // For the unit test this.conf = new OzoneConfiguration(); } @@ -120,4 +121,12 @@ public ConfigurationSource getConfiguration() { public void close() throws IOException { proxy.close(); } + + /** + * Return the ClientProtocol associated with the Ozone Cluster. + * @return ClientProtocol + */ + public ClientProtocol getProxy() { + return proxy; + } } diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java index a5d1dedb5c00..b5937647cc18 100644 --- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java +++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java @@ -1976,8 +1976,8 @@ public Cache getKeyProviderCache() { @Override public OzoneKey headObject(String volumeName, String bucketName, String keyName) throws IOException { - Preconditions.checkNotNull(volumeName); - Preconditions.checkNotNull(bucketName); + verifyVolumeName(volumeName); + verifyBucketName(bucketName); Preconditions.checkNotNull(keyName); OmKeyArgs keyArgs = new OmKeyArgs.Builder() .setVolumeName(volumeName) diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/objecthead.robot b/hadoop-ozone/dist/src/main/smoketest/s3/objecthead.robot new file mode 100644 index 000000000000..90d07ed94267 --- /dev/null +++ b/hadoop-ozone/dist/src/main/smoketest/s3/objecthead.robot @@ -0,0 +1,47 @@ +# 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. + +*** Settings *** +Documentation S3 gateway test with aws cli +Library OperatingSystem +Library String +Resource ../commonlib.robot +Resource commonawslib.robot +Test Timeout 5 minutes +Suite Setup Setup s3 tests + +*** Variables *** +${ENDPOINT_URL} http://s3g:9878 +${OZONE_TEST} true +${BUCKET} generated + +*** Test Cases *** + +Head existing object + Execute echo "Randomtext" > /tmp/testfile + ${result} = Execute AWSS3APICli and checkrc put-object --bucket ${BUCKET} --key ${PREFIX}/headobject/key=value/f1 --body /tmp/testfile 0 + + ${result} = Execute AWSS3APICli and checkrc head-object --bucket ${BUCKET} --key ${PREFIX}/headobject/key=value/f1 0 + ${result} = Execute AWSS3APICli and checkrc delete-object --bucket ${BUCKET} --key ${PREFIX}/headobject/key=value/f1 0 + +Head object in non existing bucket + ${result} = Execute AWSS3APICli and checkrc head-object --bucket ${BUCKET}-non-existent --key ${PREFIX}/headobject/key=value/f1 255 + Should contain ${result} 404 + Should contain ${result} Not Found + +Head non existing key + ${result} = Execute AWSS3APICli and checkrc head-object --bucket ${BUCKET} --key ${PREFIX}/non-existent 255 + Should contain ${result} 404 + Should contain ${result} Not Found \ No newline at end of file diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/s3_compatbility_check.sh b/hadoop-ozone/dist/src/main/smoketest/s3/s3_compatbility_check.sh index 289e2b0e24fa..b9a4c68587d6 100755 --- a/hadoop-ozone/dist/src/main/smoketest/s3/s3_compatbility_check.sh +++ b/hadoop-ozone/dist/src/main/smoketest/s3/s3_compatbility_check.sh @@ -81,6 +81,7 @@ run_robot_test objectputget run_robot_test objectdelete run_robot_test objectcopy run_robot_test objectmultidelete +run_robot_test objecthead run_robot_test MultipartUpload rebot --outputdir results/ results/*.xml diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java index b3f4d914e33d..d97bfd75b186 100644 --- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java +++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/EndpointBase.java @@ -38,6 +38,7 @@ import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneVolume; +import org.apache.hadoop.ozone.client.protocol.ClientProtocol; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes; import org.apache.hadoop.ozone.om.protocol.S3Auth; @@ -284,6 +285,10 @@ public OzoneClient getClient() { return client; } + protected ClientProtocol getClientProtocol() { + return getClient().getProxy(); + } + @VisibleForTesting public S3GatewayMetrics getMetrics() { return S3GatewayMetrics.create(); diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java index 3ef48ef1557c..1c2fa4854339 100644 --- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java +++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java @@ -64,6 +64,7 @@ import org.apache.hadoop.ozone.client.OzoneKey; import org.apache.hadoop.ozone.client.OzoneKeyDetails; import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts; +import org.apache.hadoop.ozone.client.OzoneVolume; import org.apache.hadoop.ozone.client.io.OzoneInputStream; import org.apache.hadoop.ozone.client.io.OzoneOutputStream; import org.apache.hadoop.ozone.om.exceptions.OMException; @@ -174,10 +175,11 @@ public Response put( String copyHeader = null, storageType = null; try { + OzoneVolume volume = getVolume(); if (uploadID != null && !uploadID.equals("")) { s3GAction = S3GAction.CREATE_MULTIPART_KEY; // If uploadID is specified, it is a request for upload part - return createMultipartKey(bucketName, keyPath, length, + return createMultipartKey(volume, bucketName, keyPath, length, partNumber, uploadID, body); } @@ -186,15 +188,16 @@ public Response put( boolean storageTypeDefault = StringUtils.isEmpty(storageType); // Normal put object - OzoneBucket bucket = getBucket(bucketName); + OzoneBucket bucket = volume.getBucket(bucketName); ReplicationConfig replicationConfig = getReplicationConfig(bucket, storageType); if (copyHeader != null) { //Copy object, as copy source available. s3GAction = S3GAction.COPY_OBJECT; - CopyObjectResponse copyObjectResponse = copyObject( - copyHeader, bucket, keyPath, replicationConfig, storageTypeDefault); + CopyObjectResponse copyObjectResponse = copyObject(volume, + copyHeader, bucketName, keyPath, replicationConfig, + storageTypeDefault); return Response.status(Status.OK).entity(copyObjectResponse).header( "Connection", "close").build(); } @@ -204,7 +207,7 @@ public Response put( body = new SignedChunksInputStream(body); } - output = bucket.createKey( + output = getClientProtocol().createKey(volume.getName(), bucketName, keyPath, length, replicationConfig, new HashMap<>()); IOUtils.copy(body, output); @@ -230,6 +233,8 @@ public Response put( throw os3Exception; } else if (ex.getResult() == ResultCodes.PERMISSION_DENIED) { throw newError(S3ErrorTable.ACCESS_DENIED, keyPath, ex); + } else if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) { + throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName, ex); } LOG.error("Exception occurred in PutObject", ex); throw ex; @@ -285,9 +290,10 @@ public Response get( partMarker, maxParts); } - OzoneBucket bucket = getBucket(bucketName); + OzoneVolume volume = getVolume(); - OzoneKeyDetails keyDetails = bucket.getKey(keyPath); + OzoneKeyDetails keyDetails = getClientProtocol().getKeyDetails( + volume.getName(), bucketName, keyPath); long length = keyDetails.getDataSize(); @@ -310,7 +316,8 @@ public Response get( if (rangeHeaderVal == null || rangeHeader.isReadFull()) { StreamingOutput output = dest -> { - try (OzoneInputStream key = bucket.readKey(keyPath)) { + try (OzoneInputStream key = getClientProtocol().getKey( + volume.getName(), bucketName, keyPath)) { IOUtils.copy(key, dest); } }; @@ -326,7 +333,8 @@ public Response get( // byte from start offset long copyLength = endOffset - startOffset + 1; StreamingOutput output = dest -> { - try (OzoneInputStream ozoneInputStream = bucket.readKey(keyPath)) { + try (OzoneInputStream ozoneInputStream = getClientProtocol().getKey( + volume.getName(), bucketName, keyPath)) { ozoneInputStream.seek(startOffset); IOUtils.copyLarge(ozoneInputStream, dest, 0, copyLength, new byte[bufferSize]); @@ -367,6 +375,8 @@ public Response get( throw newError(S3ErrorTable.NO_SUCH_KEY, keyPath, ex); } else if (ex.getResult() == ResultCodes.PERMISSION_DENIED) { throw newError(S3ErrorTable.ACCESS_DENIED, keyPath, ex); + } else if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) { + throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName, ex); } else { throw ex; } @@ -411,7 +421,9 @@ public Response head( OzoneKey key; try { - key = getBucket(bucketName).headObject(keyPath); + OzoneVolume volume = getVolume(); + key = getClientProtocol().headObject(volume.getName(), + bucketName, keyPath); // TODO: return the specified range bytes of this object. } catch (OMException ex) { AUDIT.logReadFailure( @@ -422,6 +434,8 @@ public Response head( return Response.status(Status.NOT_FOUND).build(); } else if (ex.getResult() == ResultCodes.PERMISSION_DENIED) { throw newError(S3ErrorTable.ACCESS_DENIED, keyPath, ex); + } else if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) { + throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName, ex); } else { throw ex; } @@ -451,14 +465,17 @@ public Response head( * @throws IOException * @throws OS3Exception */ - private Response abortMultipartUpload(String bucket, String key, String - uploadId) throws IOException, OS3Exception { + private Response abortMultipartUpload(OzoneVolume volume, String bucket, + String key, String uploadId) + throws IOException, OS3Exception { try { - OzoneBucket ozoneBucket = getBucket(bucket); - ozoneBucket.abortMultipartUpload(key, uploadId); + getClientProtocol().abortMultipartUpload(volume.getName(), bucket, + key, uploadId); } catch (OMException ex) { if (ex.getResult() == ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR) { throw newError(S3ErrorTable.NO_SUCH_UPLOAD, uploadId, ex); + } else if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) { + throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucket, ex); } throw ex; } @@ -488,13 +505,13 @@ public Response delete( S3GAction s3GAction = S3GAction.DELETE_KEY; try { + OzoneVolume volume = getVolume(); if (uploadId != null && !uploadId.equals("")) { s3GAction = S3GAction.ABORT_MULTIPART_UPLOAD; - return abortMultipartUpload(bucketName, keyPath, uploadId); + return abortMultipartUpload(volume, bucketName, keyPath, uploadId); } - OzoneBucket bucket = getBucket(bucketName); - bucket.getKey(keyPath); - bucket.deleteKey(keyPath); + getClientProtocol().deleteKey(volume.getName(), bucketName, + keyPath, false); } catch (OMException ex) { AUDIT.logWriteFailure( buildAuditMessageForFailure(s3GAction, getAuditParameters(), ex)); @@ -620,7 +637,7 @@ public Response completeMultipartUpload(@PathParam("bucket") String bucket, CompleteMultipartUploadRequest multipartUploadRequest) throws IOException, OS3Exception { S3GAction s3GAction = S3GAction.COMPLETE_MULTIPART_UPLOAD; - OzoneBucket ozoneBucket = getBucket(bucket); + OzoneVolume volume = getVolume(); // Using LinkedHashMap to preserve ordering of parts list. Map partsMap = new LinkedHashMap<>(); List partList = @@ -635,8 +652,9 @@ public Response completeMultipartUpload(@PathParam("bucket") String bucket, LOG.debug("Parts map {}", partsMap); } - omMultipartUploadCompleteInfo = ozoneBucket.completeMultipartUpload( - key, uploadID, partsMap); + omMultipartUploadCompleteInfo = getClientProtocol() + .completeMultipartUpload(volume.getName(), bucket, key, uploadID, + partsMap); CompleteMultipartUploadResponse completeMultipartUploadResponse = new CompleteMultipartUploadResponse(); completeMultipartUploadResponse.setBucket(bucket); @@ -676,6 +694,8 @@ public Response completeMultipartUpload(@PathParam("bucket") String bucket, "considered as Unix Paths. A directory already exists with a " + "given KeyName caused failure for MPU"); throw os3Exception; + } else if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) { + throw newError(S3ErrorTable.NO_SUCH_BUCKET, bucket, ex); } LOG.error("Error in Complete Multipart Upload Request for bucket: {}, " + ", key: {}", bucket, key, ex); @@ -687,12 +707,11 @@ public Response completeMultipartUpload(@PathParam("bucket") String bucket, } } - private Response createMultipartKey(String bucket, String key, long length, - int partNumber, String uploadID, - InputStream body) + private Response createMultipartKey(OzoneVolume volume, String bucket, + String key, long length, int partNumber, + String uploadID, InputStream body) throws IOException, OS3Exception { try { - OzoneBucket ozoneBucket = getBucket(bucket); String copyHeader; OzoneOutputStream ozoneOutputStream = null; @@ -702,8 +721,8 @@ private Response createMultipartKey(String bucket, String key, long length, } try { - ozoneOutputStream = ozoneBucket.createMultipartKey( - key, length, partNumber, uploadID); + ozoneOutputStream = getClientProtocol().createMultipartKey( + volume.getName(), bucket, key, length, partNumber, uploadID); copyHeader = headers.getHeaderString(COPY_SOURCE_HEADER); if (copyHeader != null) { Pair result = parseSourceHeader(copyHeader); @@ -711,8 +730,9 @@ private Response createMultipartKey(String bucket, String key, long length, String sourceBucket = result.getLeft(); String sourceKey = result.getRight(); - Long sourceKeyModificationTime = getBucket(sourceBucket). - getKey(sourceKey).getModificationTime().toEpochMilli(); + Long sourceKeyModificationTime = getClientProtocol().getKeyDetails( + volume.getName(), sourceBucket, sourceKey) + .getModificationTime().toEpochMilli(); String copySourceIfModifiedSince = headers.getHeaderString(COPY_SOURCE_IF_MODIFIED_SINCE); String copySourceIfUnmodifiedSince = @@ -722,8 +742,8 @@ private Response createMultipartKey(String bucket, String key, long length, throw newError(PRECOND_FAILED, sourceBucket + "/" + sourceKey); } - try (OzoneInputStream sourceObject = - getBucket(sourceBucket).readKey(sourceKey)) { + try (OzoneInputStream sourceObject = getClientProtocol().getKey( + volume.getName(), sourceBucket, sourceKey)) { String range = headers.getHeaderString(COPY_SOURCE_HEADER_RANGE); @@ -842,17 +862,19 @@ public void setHeaders(HttpHeaders headers) { this.headers = headers; } - static void copy(InputStream src, long srcKeyLen, - String destKey, OzoneBucket destBucket, + void copy(OzoneVolume volume, InputStream src, long srcKeyLen, + String destKey, String destBucket, ReplicationConfig replication) throws IOException { - try (OzoneOutputStream dest = destBucket.createKey( - destKey, srcKeyLen, replication, new HashMap<>())) { + try (OzoneOutputStream dest = getClientProtocol().createKey( + volume.getName(), destBucket, destKey, srcKeyLen, + replication, new HashMap<>())) { IOUtils.copy(src, dest); } } - private CopyObjectResponse copyObject(String copyHeader, - OzoneBucket destBucket, + private CopyObjectResponse copyObject(OzoneVolume volume, + String copyHeader, + String destBucket, String destkey, ReplicationConfig replicationConfig, boolean storageTypeDefault) @@ -865,7 +887,7 @@ private CopyObjectResponse copyObject(String copyHeader, try { // Checking whether we trying to copying to it self. - if (sourceBucket.equals(destBucket.getName()) && sourceKey + if (sourceBucket.equals(destBucket) && sourceKey .equals(destkey)) { // When copying to same storage type when storage type is provided, // we should not throw exception, as aws cli checks if any of the @@ -890,17 +912,17 @@ private CopyObjectResponse copyObject(String copyHeader, } } - - OzoneBucket sourceOzoneBucket = getBucket(sourceBucket); - - OzoneKeyDetails sourceKeyDetails = sourceOzoneBucket.getKey(sourceKey); + OzoneKeyDetails sourceKeyDetails = getClientProtocol().getKeyDetails( + volume.getName(), sourceBucket, sourceKey); long sourceKeyLen = sourceKeyDetails.getDataSize(); - try (OzoneInputStream src = sourceOzoneBucket.readKey(sourceKey)) { - copy(src, sourceKeyLen, destkey, destBucket, replicationConfig); + try (OzoneInputStream src = getClientProtocol().getKey(volume.getName(), + sourceBucket, sourceKey)) { + copy(volume, src, sourceKeyLen, destkey, destBucket, replicationConfig); } - final OzoneKeyDetails destKeyDetails = destBucket.getKey(destkey); + final OzoneKeyDetails destKeyDetails = getClientProtocol().getKeyDetails( + volume.getName(), destBucket, destkey); getMetrics().incCopyObjectSuccess(); CopyObjectResponse copyObjectResponse = new CopyObjectResponse(); diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java new file mode 100644 index 000000000000..f0234889864e --- /dev/null +++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ClientProtocolStub.java @@ -0,0 +1,573 @@ +/* + * 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.client; + +import org.apache.hadoop.crypto.key.KeyProvider; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationFactor; +import org.apache.hadoop.hdds.client.ReplicationType; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.StorageType; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.ozone.OzoneAcl; +import org.apache.hadoop.ozone.client.io.OzoneInputStream; +import org.apache.hadoop.ozone.client.io.OzoneOutputStream; +import org.apache.hadoop.ozone.client.protocol.ClientProtocol; +import org.apache.hadoop.ozone.om.helpers.DeleteTenantState; +import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo; +import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo; +import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo; +import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs; +import org.apache.hadoop.ozone.om.helpers.OzoneFileStatus; +import org.apache.hadoop.ozone.om.helpers.RepeatedOmKeyInfo; +import org.apache.hadoop.ozone.om.helpers.S3SecretValue; +import org.apache.hadoop.ozone.om.helpers.S3VolumeContext; +import org.apache.hadoop.ozone.om.helpers.TenantStateList; +import org.apache.hadoop.ozone.om.helpers.TenantUserInfoValue; +import org.apache.hadoop.ozone.om.helpers.TenantUserList; +import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; +import org.apache.hadoop.ozone.om.protocol.S3Auth; +import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos; +import org.apache.hadoop.ozone.security.OzoneTokenIdentifier; +import org.apache.hadoop.ozone.security.acl.OzoneObj; +import org.apache.hadoop.security.token.Token; + +import java.io.IOException; +import java.net.URI; +import java.util.List; +import java.util.Map; + +/** + * ClientProtocol implementation with in-memory state. + */ +public class ClientProtocolStub implements ClientProtocol { + private final ObjectStoreStub objectStoreStub; + + public ClientProtocolStub(ObjectStoreStub objectStoreStub) { + this.objectStoreStub = objectStoreStub; + } + + @Override + public List getOmRoleInfos() + throws IOException { + return null; + } + + @Override + public void createVolume(String volumeName) throws IOException { + } + + @Override + public void createVolume(String volumeName, VolumeArgs args) + throws IOException { + } + + @Override + public boolean setVolumeOwner(String volumeName, String owner) + throws IOException { + return false; + } + + @Override + public void setVolumeQuota(String volumeName, long quotaInNamespace, + long quotaInBytes) throws IOException { + } + + @Override + public OzoneVolume getVolumeDetails(String volumeName) throws IOException { + return objectStoreStub.getVolume(volumeName); + } + + @Override + public S3VolumeContext getS3VolumeContext() throws IOException { + return null; + } + + @Override + public OzoneVolume buildOzoneVolume(OmVolumeArgs volume) { + return null; + } + + @Override + public boolean checkVolumeAccess(String volumeName, OzoneAcl acl) + throws IOException { + return false; + } + + @Override + public void deleteVolume(String volumeName) throws IOException { + + } + + @Override + public List listVolumes(String volumePrefix, String prevVolume, + int maxListResult) throws IOException { + return null; + } + + @Override + public List listVolumes(String user, String volumePrefix, + String prevVolume, int maxListResult) + throws IOException { + return null; + } + + @Override + public void createBucket(String volumeName, String bucketName) + throws IOException { + + } + + @Override + public void createBucket(String volumeName, String bucketName, + BucketArgs bucketArgs) throws IOException { + + } + + @Override + public void setBucketVersioning(String volumeName, String bucketName, + Boolean versioning) throws IOException { + + } + + @Override + public void setBucketStorageType(String volumeName, String bucketName, + StorageType storageType) throws IOException { + + } + + @Override + public void deleteBucket(String volumeName, String bucketName) + throws IOException { + + } + + @Override + public void checkBucketAccess(String volumeName, String bucketName) + throws IOException { + + } + + @Override + public OzoneBucket getBucketDetails(String volumeName, String bucketName) + throws IOException { + return null; + } + + @Override + public List listBuckets(String volumeName, String bucketPrefix, + String prevBucket, int maxListResult) + throws IOException { + return null; + } + + @Override + public OzoneOutputStream createKey(String volumeName, String bucketName, + String keyName, long size, + ReplicationType type, + ReplicationFactor factor, + Map metadata) + throws IOException { + return getBucket(volumeName, bucketName).createKey(keyName, size); + } + + @Override + public OzoneOutputStream createKey(String volumeName, String bucketName, + String keyName, long size, + ReplicationConfig replicationConfig, + Map metadata) + throws IOException { + return getBucket(volumeName, bucketName) + .createKey(keyName, size, replicationConfig, metadata); + } + + @Override + public OzoneInputStream getKey(String volumeName, String bucketName, + String keyName) throws IOException { + return getBucket(volumeName, bucketName).readKey(keyName); + } + + private OzoneBucket getBucket(String volumeName, String bucketName) + throws IOException { + return objectStoreStub.getVolume(volumeName).getBucket(bucketName); + } + + @Override + public void deleteKey(String volumeName, String bucketName, String keyName, + boolean recursive) throws IOException { + getBucket(volumeName, bucketName).deleteKey(keyName); + } + + @Override + public void deleteKeys(String volumeName, String bucketName, + List keyNameList) throws IOException { + + } + + @Override + public void renameKey(String volumeName, String bucketName, + String fromKeyName, String toKeyName) + throws IOException { + + } + + @Override + public void renameKeys(String volumeName, String bucketName, + Map keyMap) throws IOException { + + } + + @Override + public List listKeys(String volumeName, String bucketName, + String keyPrefix, String prevKey, + int maxListResult) throws IOException { + return null; + } + + @Override + public List listTrash(String volumeName, String bucketName, + String startKeyName, + String keyPrefix, int maxKeys) + throws IOException { + return null; + } + + @Override + public boolean recoverTrash(String volumeName, String bucketName, + String keyName, String destinationBucket) + throws IOException { + return false; + } + + @Override + public OzoneKeyDetails getKeyDetails(String volumeName, String bucketName, + String keyName) throws IOException { + return getBucket(volumeName, bucketName).getKey(keyName); + } + + @Override + public void close() throws IOException { + + } + + @Override + public OmMultipartInfo initiateMultipartUpload(String volumeName, + String bucketName, + String keyName, + ReplicationType type, + ReplicationFactor factor) + throws IOException { + return null; + } + + @Override + public OmMultipartInfo initiateMultipartUpload(String volumeName, + String bucketName, String keyName, ReplicationConfig replicationConfig) + throws IOException { + return getBucket(volumeName, bucketName) + .initiateMultipartUpload(keyName, replicationConfig); + } + + @Override + public OzoneOutputStream createMultipartKey(String volumeName, + String bucketName, String keyName, + long size, int partNumber, + String uploadID) + throws IOException { + return getBucket(volumeName, bucketName).createMultipartKey(keyName, size, + partNumber, uploadID); + } + + @Override + public OmMultipartUploadCompleteInfo completeMultipartUpload( + String volumeName, String bucketName, String keyName, String uploadID, + Map partsMap) throws IOException { + return getBucket(volumeName, bucketName) + .completeMultipartUpload(keyName, uploadID, partsMap); + } + + @Override + public void abortMultipartUpload(String volumeName, String bucketName, + String keyName, String uploadID) + throws IOException { + getBucket(volumeName, bucketName).abortMultipartUpload(keyName, uploadID); + } + + @Override + public OzoneMultipartUploadPartListParts listParts(String volumeName, + String bucketName, + String keyName, + String uploadID, + int partNumberMarker, + int maxParts) + throws IOException { + return getBucket(volumeName, bucketName) + .listParts(keyName, uploadID, partNumberMarker, maxParts); + } + + @Override + public OzoneMultipartUploadList listMultipartUploads(String volumename, + String bucketName, + String prefix) + throws IOException { + return null; + } + + @Override + public Token getDelegationToken(Text renewer) + throws IOException { + return null; + } + + @Override + public long renewDelegationToken(Token token) + throws IOException { + return 0; + } + + @Override + public void cancelDelegationToken(Token token) + throws IOException { + + } + + @Override + public S3SecretValue getS3Secret(String kerberosID) throws IOException { + return null; + } + + @Override + public S3SecretValue getS3Secret(String kerberosID, boolean createIfNotExist) + throws IOException { + return null; + } + + @Override + public S3SecretValue setS3Secret(String accessId, String secretKey) + throws IOException { + return null; + } + + @Override + public void revokeS3Secret(String kerberosID) throws IOException { + + } + + @Override + public void createTenant(String tenantId) throws IOException { + + } + + @Override + public void createTenant(String tenantId, TenantArgs tenantArgs) + throws IOException { + + } + + @Override + public DeleteTenantState deleteTenant(String tenantId) throws IOException { + return null; + } + + @Override + public S3SecretValue tenantAssignUserAccessId(String username, + String tenantId, + String accessId) + throws IOException { + return null; + } + + @Override + public void tenantRevokeUserAccessId(String accessId) throws IOException { + + } + + @Override + public void tenantAssignAdmin(String accessId, String tenantId, + boolean delegated) throws IOException { + + } + + @Override + public void tenantRevokeAdmin(String accessId, String tenantId) + throws IOException { + + } + + @Override + public TenantUserInfoValue tenantGetUserInfo(String userPrincipal) + throws IOException { + return null; + } + + @Override + public TenantUserList listUsersInTenant(String tenantId, String prefix) + throws IOException { + return null; + } + + @Override + public TenantStateList listTenant() throws IOException { + return null; + } + + @Override + public KeyProvider getKeyProvider() throws IOException { + return null; + } + + @Override + public URI getKeyProviderUri() throws IOException { + return null; + } + + @Override + public String getCanonicalServiceName() { + return null; + } + + @Override + public OzoneFileStatus getOzoneFileStatus(String volumeName, + String bucketName, String keyName) + throws IOException { + return null; + } + + @Override + public void createDirectory(String volumeName, String bucketName, + String keyName) throws IOException { + + } + + @Override + public OzoneInputStream readFile(String volumeName, String bucketName, + String keyName) throws IOException { + return null; + } + + @Override + public OzoneOutputStream createFile(String volumeName, String bucketName, + String keyName, long size, + ReplicationType type, + ReplicationFactor factor, + boolean overWrite, boolean recursive) + throws IOException { + return null; + } + + @Override + public OzoneOutputStream createFile(String volumeName, String bucketName, + String keyName, long size, + ReplicationConfig replicationConfig, + boolean overWrite, boolean recursive) + throws IOException { + return null; + } + + @Override + public List listStatus(String volumeName, String bucketName, + String keyName, boolean recursive, + String startKey, long numEntries) + throws IOException { + return null; + } + + @Override + public List listStatus(String volumeName, String bucketName, + String keyName, boolean recursive, + String startKey, long numEntries, + boolean allowPartialPrefixes) + throws IOException { + return null; + } + + @Override + public boolean addAcl(OzoneObj obj, OzoneAcl acl) throws IOException { + return false; + } + + @Override + public boolean removeAcl(OzoneObj obj, OzoneAcl acl) throws IOException { + return false; + } + + @Override + public boolean setAcl(OzoneObj obj, List acls) throws IOException { + return false; + } + + @Override + public List getAcl(OzoneObj obj) throws IOException { + return null; + } + + @Override + public OzoneManagerProtocol getOzoneManagerClient() { + return null; + } + + @Override + public void setBucketQuota(String volumeName, String bucketName, + long quotaInNamespace, long quotaInBytes) + throws IOException { + + } + + @Override + public void setReplicationConfig(String volumeName, String bucketName, + ReplicationConfig replicationConfig) + throws IOException { + + } + + @Override + public OzoneKey headObject(String volumeName, String bucketName, + String keyName) throws IOException { + return getBucket(volumeName, bucketName).headObject(keyName); + } + + @Override + public void setThreadLocalS3Auth(S3Auth s3Auth) { + + } + + @Override + public S3Auth getThreadLocalS3Auth() { + return null; + } + + @Override + public void clearThreadLocalS3Auth() { + + } + + @Override + public boolean setBucketOwner(String volumeName, String bucketName, + String owner) throws IOException { + return false; + } + + @Override + public Map> getKeysEveryReplicas( + String volumeName, String bucketName, String keyName) throws IOException { + return null; + } +} diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java index 5591ff6aa2f0..5124cad5264b 100644 --- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java +++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/ObjectStoreStub.java @@ -160,4 +160,5 @@ public void deleteS3Bucket(String s3BucketName) throws public void setBucketEmptyStatus(String bucketName, boolean status) { bucketEmptyStatus.computeIfPresent(bucketName, (k, v) -> status); } + } diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneClientStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneClientStub.java index e8237cadb401..64f515060b49 100644 --- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneClientStub.java +++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneClientStub.java @@ -23,9 +23,12 @@ * In-memory OzoneClient for testing. */ public class OzoneClientStub extends OzoneClient { - public OzoneClientStub() { - super(new ObjectStoreStub()); + this(new ObjectStoreStub()); + } + + public OzoneClientStub(ObjectStoreStub objectStoreStub) { + super(objectStoreStub, new ClientProtocolStub(objectStoreStub)); } @Override diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java index 4ca0ef0e45ec..0be5b917921a 100644 --- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java +++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestPermissionCheck.java @@ -25,6 +25,7 @@ import org.apache.hadoop.ozone.client.OzoneBucket; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneVolume; +import org.apache.hadoop.ozone.client.protocol.ClientProtocol; import org.apache.hadoop.ozone.om.exceptions.OMException; import org.apache.hadoop.ozone.s3.exception.OS3Exception; import org.junit.Assert; @@ -41,6 +42,7 @@ import static java.net.HttpURLConnection.HTTP_FORBIDDEN; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doThrow; @@ -48,12 +50,15 @@ /** * Test operation permission check result. + * Test-cases in this class verify that permission denied errors from OM + * are handled correctly by S3 endpoints. */ public class TestPermissionCheck { private OzoneConfiguration conf; private OzoneClient client; - private ObjectStore objectStore; + private ClientProtocol clientProtocol; private OzoneBucket bucket; + private ObjectStore objectStore; private OzoneVolume volume; private OMException exception; private HttpHeaders headers; @@ -67,11 +72,14 @@ public void setup() { objectStore = Mockito.mock(ObjectStore.class); bucket = Mockito.mock(OzoneBucket.class); volume = Mockito.mock(OzoneVolume.class); + when(volume.getName()).thenReturn("s3Volume"); exception = new OMException("Permission Denied", OMException.ResultCodes.PERMISSION_DENIED); Mockito.when(client.getObjectStore()).thenReturn(objectStore); Mockito.when(client.getConfiguration()).thenReturn(conf); headers = Mockito.mock(HttpHeaders.class); + clientProtocol = Mockito.mock(ClientProtocol.class); + Mockito.when(client.getProxy()).thenReturn(clientProtocol); } /** @@ -246,8 +254,10 @@ public void testSetAcl() throws Exception { */ @Test public void testGetKey() throws IOException { - Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket); - doThrow(exception).when(bucket).getKey(anyString()); + Mockito.when(objectStore.getS3Volume()).thenReturn(volume); + Mockito.when(client.getProxy()).thenReturn(clientProtocol); + doThrow(exception).when(clientProtocol) + .getKeyDetails(anyString(), anyString(), anyString()); ObjectEndpoint objectEndpoint = new ObjectEndpoint(); objectEndpoint.setClient(client); objectEndpoint.setHeaders(headers); @@ -266,9 +276,10 @@ public void testGetKey() throws IOException { @Test public void testPutKey() throws IOException { - Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket); - doThrow(exception).when(bucket) - .createKey(anyString(), anyLong(), any(), any()); + Mockito.when(objectStore.getS3Volume()).thenReturn(volume); + Mockito.when(volume.getBucket("bucketName")).thenReturn(bucket); + doThrow(exception).when(clientProtocol).createKey( + anyString(), anyString(), anyString(), anyLong(), any(), any()); ObjectEndpoint objectEndpoint = new ObjectEndpoint(); objectEndpoint.setClient(client); objectEndpoint.setHeaders(headers); @@ -285,8 +296,9 @@ public void testPutKey() throws IOException { @Test public void testDeleteKey() throws IOException { - Mockito.when(objectStore.getS3Bucket(anyString())).thenReturn(bucket); - doThrow(exception).when(bucket).deleteKey(anyString()); + Mockito.when(objectStore.getS3Volume()).thenReturn(volume); + doThrow(exception).when(clientProtocol).deleteKey(anyString(), anyString(), + anyString(), anyBoolean()); ObjectEndpoint objectEndpoint = new ObjectEndpoint(); objectEndpoint.setClient(client); objectEndpoint.setHeaders(headers);