You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ae...@apache.org on 2019/02/15 06:59:01 UTC
[hadoop] branch trunk updated: HDDS-1068. Improve the error
propagation for ozone sh. Contributed by Elek, Marton.
This is an automated email from the ASF dual-hosted git repository.
aengineer pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/trunk by this push:
new 0395f22 HDDS-1068. Improve the error propagation for ozone sh. Contributed by Elek, Marton.
0395f22 is described below
commit 0395f22145d90d38895a7a3e220a15718b1e2399
Author: Anu Engineer <ae...@apache.org>
AuthorDate: Thu Feb 14 22:51:46 2019 -0800
HDDS-1068. Improve the error propagation for ozone sh.
Contributed by Elek, Marton.
---
.../org/apache/hadoop/hdds/cli/GenericCli.java | 5 +-
...lockLocationProtocolClientSideTranslatorPB.java | 65 +-
.../apache/hadoop/ozone/client/ObjectStore.java | 10 +-
.../hadoop/ozone/om/exceptions/OMException.java | 98 +-
...OzoneManagerProtocolClientSideTranslatorPB.java | 327 ++---
.../src/main/proto/OzoneManagerProtocol.proto | 62 +-
.../ozone/om/exceptions/TestResultCodes.java | 49 +
.../org/apache/hadoop/ozone/OzoneTestUtils.java | 20 +-
.../hadoop/ozone/TestSecureOzoneCluster.java | 5 +-
.../ozone/client/rest/TestOzoneRestClient.java | 499 -------
.../hadoop/ozone/client/rest/package-info.java | 23 -
.../client/rpc/TestOzoneRpcClientAbstract.java | 216 ++-
.../org/apache/hadoop/ozone/om/TestOmAcls.java | 27 +-
.../apache/hadoop/ozone/om/TestOzoneManager.java | 208 ++-
.../hadoop/ozone/ozShell/TestOzoneShell.java | 86 +-
.../apache/hadoop/ozone/web/client/TestKeys.java | 82 +-
.../apache/hadoop/ozone/web/client/TestVolume.java | 57 +-
.../hadoop/ozone/web/client/TestVolumeRatis.java | 2 +-
.../apache/hadoop/ozone/om/BucketManagerImpl.java | 12 +-
.../org/apache/hadoop/ozone/om/KeyManagerImpl.java | 53 +-
.../hadoop/ozone/om/OmMetadataManagerImpl.java | 18 +-
.../hadoop/ozone/om/S3BucketManagerImpl.java | 5 +-
.../apache/hadoop/ozone/om/VolumeManagerImpl.java | 18 +-
.../protocolPB/OzoneManagerRequestHandler.java | 1378 +++++++++-----------
.../org/apache/hadoop/ozone/web/ozShell/Shell.java | 17 +
.../hadoop/ozone/om/TestBucketManagerImpl.java | 10 +-
.../hadoop/ozone/s3/endpoint/BucketEndpoint.java | 18 +-
.../hadoop/ozone/s3/endpoint/EndpointBase.java | 30 +-
.../hadoop/ozone/s3/endpoint/ObjectEndpoint.java | 62 +-
.../hadoop/ozone/client/ObjectStoreStub.java | 28 +-
.../hadoop/ozone/client/OzoneBucketStub.java | 18 +-
31 files changed, 1365 insertions(+), 2143 deletions(-)
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
index 849aba0..79ef711 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/cli/GenericCli.java
@@ -61,10 +61,11 @@ public class GenericCli implements Callable<Void>, GenericParentCommand {
cmd.parseWithHandler(new RunLast(), argv);
}
- private void printError(Throwable error) {
+ protected void printError(Throwable error) {
//message could be null in case of NPE. This is unexpected so we can
//print out the stack trace.
- if (verbose || error.getMessage() == null) {
+ if (verbose || error.getMessage() == null
+ || error.getMessage().length() == 0) {
error.printStackTrace(System.err);
} else {
System.err.println(error.getMessage().split("\n")[0]);
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
index e00c634..bf5a55b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ScmBlockLocationProtocolClientSideTranslatorPB.java
@@ -16,37 +16,32 @@
*/
package org.apache.hadoop.hdds.scm.protocolPB;
-import com.google.common.base.Preconditions;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.ServiceException;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdds.client.ContainerBlockID;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.DeleteScmKeyBlocksRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.DeleteScmKeyBlocksResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.KeyBlocks;
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
- .AllocateScmBlockRequestProto;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
- .AllocateScmBlockResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
- .DeleteScmKeyBlocksRequestProto;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
- .DeleteScmKeyBlocksResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
- .KeyBlocks;
-import org.apache.hadoop.ipc.ProtobufHelper;
import org.apache.hadoop.ipc.ProtocolTranslator;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.stream.Collectors;
+import com.google.common.base.Preconditions;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.ServiceException;
/**
* This class is the client-side translator to translate the requests made on
@@ -94,7 +89,7 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
try {
response = rpcProxy.allocateScmBlock(NULL_RPC_CONTROLLER, request);
} catch (ServiceException e) {
- throw ProtobufHelper.getRemoteException(e);
+ throw transformServiceException(e);
}
if (response.getErrorCode() !=
AllocateScmBlockResponseProto.Error.success) {
@@ -128,7 +123,7 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
try {
resp = rpcProxy.deleteScmKeyBlocks(NULL_RPC_CONTROLLER, request);
} catch (ServiceException e) {
- throw ProtobufHelper.getRemoteException(e);
+ throw transformServiceException(e);
}
List<DeleteBlockGroupResult> results =
new ArrayList<>(resp.getResultsCount());
@@ -140,6 +135,30 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
return results;
}
+ private IOException transformServiceException(
+ ServiceException se) throws IOException {
+ //TODO SCM has no perfect way to return with business exceptions. All
+ //the exceptions will be mapped to ServiceException.
+ //ServiceException is handled in a special way in hadoop rpc: the message
+ //contains the whole stack trace which is not required for the business
+ //exception. As of now I remove the stack trace (use first line only).
+ //Long term we need a proper way of the exception propagation.
+ Throwable cause = se.getCause();
+ if (cause == null) {
+ return new IOException(
+ new ServiceException(useFirstLine(se.getMessage()), se.getCause()));
+ }
+ return new IOException(useFirstLine(cause.getMessage()), cause.getCause());
+ }
+
+ private String useFirstLine(String message) {
+ if (message == null) {
+ return null;
+ } else {
+ return message.split("\n")[0];
+ }
+ }
+
/**
* Gets the cluster Id and Scm Id from SCM.
* @return ScmInfo
@@ -153,7 +172,7 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
try {
resp = rpcProxy.getScmInfo(NULL_RPC_CONTROLLER, request);
} catch (ServiceException e) {
- throw ProtobufHelper.getRemoteException(e);
+ throw transformServiceException(e);
}
ScmInfo.Builder builder = new ScmInfo.Builder()
.setClusterId(resp.getClusterId())
diff --git a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
index 8370bbc..a6fb818 100644
--- a/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
+++ b/hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/ObjectStore.java
@@ -30,6 +30,8 @@ import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.tracing.TracingUtil;
import org.apache.hadoop.io.Text;
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.helpers.S3SecretValue;
import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
import org.apache.hadoop.security.UserGroupInformation;
@@ -372,12 +374,14 @@ public class ObjectStore {
try {
return proxy.listS3Buckets(userName, bucketPrefix, prevBucket,
listCacheSize);
- } catch (IOException e) {
- if (e.getMessage().contains("VOLUME_NOT_FOUND")) {
- return new ArrayList<OzoneBucket>();
+ } catch (OMException e) {
+ if (e.getResult() == ResultCodes.VOLUME_NOT_FOUND) {
+ return new ArrayList<>();
} else {
throw new RuntimeException(e);
}
+ } catch (IOException e) {
+ throw new RuntimeException(e);
}
}
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
index 3581ed8..609000c 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
@@ -62,7 +62,7 @@ public class OMException extends IOException {
* @since 1.6
*/
public OMException(String message, Throwable cause,
- OMException.ResultCodes result) {
+ OMException.ResultCodes result) {
super(message, cause);
this.result = result;
}
@@ -92,45 +92,91 @@ public class OMException extends IOException {
return result;
}
+ @Override
+ public String toString() {
+ return result + " " + super.toString();
+ }
/**
* Error codes to make it easy to decode these exceptions.
*/
public enum ResultCodes {
- FAILED_TOO_MANY_USER_VOLUMES,
- FAILED_VOLUME_ALREADY_EXISTS,
- FAILED_VOLUME_NOT_FOUND,
- FAILED_VOLUME_NOT_EMPTY,
- FAILED_USER_NOT_FOUND,
- FAILED_BUCKET_ALREADY_EXISTS,
- FAILED_BUCKET_NOT_FOUND,
- FAILED_BUCKET_NOT_EMPTY,
- FAILED_KEY_ALREADY_EXISTS,
- FAILED_KEY_NOT_FOUND,
- FAILED_KEY_ALLOCATION,
- FAILED_KEY_DELETION,
- FAILED_KEY_RENAME,
- FAILED_INVALID_KEY_NAME,
- FAILED_METADATA_ERROR,
+
+ OK,
+
+ VOLUME_NOT_UNIQUE,
+
+ VOLUME_NOT_FOUND,
+
+ VOLUME_NOT_EMPTY,
+
+ VOLUME_ALREADY_EXISTS,
+
+ USER_NOT_FOUND,
+
+ USER_TOO_MANY_VOLUMES,
+
+ BUCKET_NOT_FOUND,
+
+ BUCKET_NOT_EMPTY,
+
+ BUCKET_ALREADY_EXISTS,
+
+ KEY_ALREADY_EXISTS,
+
+ KEY_NOT_FOUND,
+
+ INVALID_KEY_NAME,
+
+ ACCESS_DENIED,
+
+ INTERNAL_ERROR,
+
+ KEY_ALLOCATION_ERROR,
+
+ KEY_DELETION_ERROR,
+
+ KEY_RENAME_ERROR,
+
+ METADATA_ERROR,
+
OM_NOT_INITIALIZED,
+
SCM_VERSION_MISMATCH_ERROR,
- SCM_IN_CHILL_MODE,
- S3_BUCKET_ALREADY_EXISTS,
+
S3_BUCKET_NOT_FOUND,
- S3_SECRET_NOT_FOUND,
- INITIATE_MULTIPART_UPLOAD_FAILED,
- NO_SUCH_MULTIPART_UPLOAD,
- UPLOAD_PART_FAILED,
+
+ S3_BUCKET_ALREADY_EXISTS,
+
+ INITIATE_MULTIPART_UPLOAD_ERROR,
+
+ MULTIPART_UPLOAD_PARTFILE_ERROR,
+
+ NO_SUCH_MULTIPART_UPLOAD_ERROR,
+
MISMATCH_MULTIPART_LIST,
+
MISSING_UPLOAD_PARTS,
- COMPLETE_MULTIPART_UPLOAD_FAILED,
+
+ COMPLETE_MULTIPART_UPLOAD_ERROR,
+
ENTITY_TOO_SMALL,
+
ABORT_MULTIPART_UPLOAD_FAILED,
- INVALID_REQUEST,
+
+ S3_SECRET_NOT_FOUND,
+
INVALID_AUTH_METHOD,
+
INVALID_TOKEN,
+
TOKEN_EXPIRED,
+
TOKEN_ERROR_OTHER,
- UNKNOWN,
- LIST_MULTIPART_UPLOAD_PARTS_FAILED;
+
+ LIST_MULTIPART_UPLOAD_PARTS_FAILED,
+
+ SCM_IN_CHILL_MODE,
+
+ INVALID_REQUEST
}
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
index 6eefcc0..cd4ac3c 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/protocolPB/OzoneManagerProtocolClientSideTranslatorPB.java
@@ -53,17 +53,13 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CancelD
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CheckVolumeAccessRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CheckVolumeAccessResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CommitKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CommitKeyResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateBucketRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateBucketResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateKeyRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateKeyResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateVolumeResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteBucketResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeyResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetDelegationTokenResponseProto;
@@ -87,7 +83,6 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Multipa
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartInfoInitiateRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartInfoInitiateResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadAbortRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadAbortResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadCompleteRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadCompleteResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadListPartsRequest;
@@ -96,14 +91,11 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMReque
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OzoneAclInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RenameKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RenameKeyResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RenewDelegationTokenResponseProto;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3BucketInfoRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3BucketInfoResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3CreateBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3CreateBucketResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3DeleteBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3DeleteBucketResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3ListBucketsRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3ListBucketsResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListRequest;
@@ -111,8 +103,6 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Service
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetBucketPropertyRequest;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetBucketPropertyResponse;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetVolumePropertyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetVolumePropertyResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo;
import org.apache.hadoop.ozone.protocolPB.OMPBHelper;
@@ -130,11 +120,11 @@ import io.opentracing.Scope;
import io.opentracing.util.GlobalTracer;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKEN_ERROR_OTHER;
-import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.UNKNOWN;
import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status.ACCESS_DENIED;
import static org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status.OK;
+
/**
* The client side implementation of OzoneManagerProtocol.
*/
@@ -240,13 +230,8 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setCreateVolumeRequest(req)
.build();
- CreateVolumeResponse resp = submitRequest(omRequest)
- .getCreateVolumeResponse();
-
- if (resp.getStatus() != OK) {
- throw new
- IOException("Volume creation failed, error:" + resp.getStatus());
- }
+ OMResponse omResponse = submitRequest(omRequest);
+ handleError(omResponse);
}
/**
@@ -266,13 +251,8 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setSetVolumePropertyRequest(req)
.build();
- SetVolumePropertyResponse resp = submitRequest(omRequest)
- .getSetVolumePropertyResponse();
-
- if (resp.getStatus() != OK) {
- throw new
- IOException("Volume owner change failed, error:" + resp.getStatus());
- }
+ OMResponse omResponse = submitRequest(omRequest);
+ handleError(omResponse);
}
/**
@@ -292,13 +272,8 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setSetVolumePropertyRequest(req)
.build();
- SetVolumePropertyResponse resp = submitRequest(omRequest)
- .getSetVolumePropertyResponse();
-
- if (resp.getStatus() != OK) {
- throw new
- IOException("Volume quota change failed, error:" + resp.getStatus());
- }
+ OMResponse omResponse = submitRequest(omRequest);
+ handleError(omResponse);
}
/**
@@ -321,16 +296,17 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setCheckVolumeAccessRequest(req)
.build();
- CheckVolumeAccessResponse resp = submitRequest(omRequest)
+ OMResponse omResponse = submitRequest(omRequest);
+ CheckVolumeAccessResponse resp = omResponse
.getCheckVolumeAccessResponse();
- if (resp.getStatus() == ACCESS_DENIED) {
+ if (omResponse.getStatus() == ACCESS_DENIED) {
return false;
- } else if (resp.getStatus() == OK) {
+ } else if (omResponse.getStatus() == OK) {
return true;
} else {
- throw new
- IOException("Check Volume Access failed, error:" + resp.getStatus());
+ handleError(omResponse);
+ return false;
}
}
@@ -350,12 +326,10 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setInfoVolumeRequest(req)
.build();
- InfoVolumeResponse resp = submitRequest(omRequest).getInfoVolumeResponse();
+ InfoVolumeResponse resp =
+ handleError(submitRequest(omRequest)).getInfoVolumeResponse();
+
- if (resp.getStatus() != OK) {
- throw new
- IOException("Info Volume failed, error:" + resp.getStatus());
- }
return OmVolumeArgs.getFromProtobuf(resp.getVolumeInfo());
}
@@ -374,13 +348,9 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setDeleteVolumeRequest(req)
.build();
- DeleteVolumeResponse resp = submitRequest(omRequest)
+ DeleteVolumeResponse resp = handleError(submitRequest(omRequest))
.getDeleteVolumeResponse();
- if (resp.getStatus() != OK) {
- throw new
- IOException("Delete Volume failed, error:" + resp.getStatus());
- }
}
/**
@@ -443,12 +413,10 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setListVolumeRequest(request)
.build();
- ListVolumeResponse resp = submitRequest(omRequest).getListVolumeResponse();
+ ListVolumeResponse resp =
+ handleError(submitRequest(omRequest)).getListVolumeResponse();
+
- if (resp.getStatus() != OK) {
- throw new IOException("List volume failed, error: "
- + resp.getStatus());
- }
return resp.getVolumeInfoList().stream()
.map(item -> OmVolumeArgs.getFromProtobuf(item))
@@ -472,13 +440,9 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setCreateBucketRequest(req)
.build();
- CreateBucketResponse resp = submitRequest(omRequest)
+ CreateBucketResponse resp = handleError(submitRequest(omRequest))
.getCreateBucketResponse();
- if (resp.getStatus() != OK) {
- throw new IOException("Bucket creation failed, error: "
- + resp.getStatus());
- }
}
/**
@@ -501,14 +465,10 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setInfoBucketRequest(req)
.build();
- InfoBucketResponse resp = submitRequest(omRequest).getInfoBucketResponse();
+ InfoBucketResponse resp =
+ handleError(submitRequest(omRequest)).getInfoBucketResponse();
- if (resp.getStatus() == OK) {
- return OmBucketInfo.getFromProtobuf(resp.getBucketInfo());
- } else {
- throw new IOException("Info Bucket failed, error: "
- + resp.getStatus());
- }
+ return OmBucketInfo.getFromProtobuf(resp.getBucketInfo());
}
/**
@@ -528,13 +488,9 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setSetBucketPropertyRequest(req)
.build();
- SetBucketPropertyResponse resp = submitRequest(omRequest)
+ SetBucketPropertyResponse resp = handleError(submitRequest(omRequest))
.getSetBucketPropertyResponse();
- if (resp.getStatus() != OK) {
- throw new IOException("Setting bucket property failed, error: "
- + resp.getStatus());
- }
}
/**
@@ -566,19 +522,15 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setListBucketsRequest(request)
.build();
- ListBucketsResponse resp = submitRequest(omRequest)
+ ListBucketsResponse resp = handleError(submitRequest(omRequest))
.getListBucketsResponse();
- if (resp.getStatus() == OK) {
- buckets.addAll(
+ buckets.addAll(
resp.getBucketInfoList().stream()
.map(OmBucketInfo::getFromProtobuf)
.collect(Collectors.toList()));
- return buckets;
- } else {
- throw new IOException("List Buckets failed, error: "
- + resp.getStatus());
- }
+ return buckets;
+
}
/**
@@ -630,13 +582,19 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setCreateKeyRequest(req)
.build();
- CreateKeyResponse resp = submitRequest(omRequest).getCreateKeyResponse();
+ CreateKeyResponse keyResponse =
+ handleError(submitRequest(omRequest)).getCreateKeyResponse();
+ return new OpenKeySession(keyResponse.getID(),
+ OmKeyInfo.getFromProtobuf(keyResponse.getKeyInfo()),
+ keyResponse.getOpenVersion());
+ }
+ private OMResponse handleError(OMResponse resp) throws OMException {
if (resp.getStatus() != OK) {
- throw new IOException("Create key failed, error:" + resp.getStatus());
+ throw new OMException(resp.getMessage(),
+ ResultCodes.values()[resp.getStatus().ordinal()]);
}
- return new OpenKeySession(resp.getID(),
- OmKeyInfo.getFromProtobuf(resp.getKeyInfo()), resp.getOpenVersion());
+ return resp;
}
@Override
@@ -655,13 +613,8 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setAllocateBlockRequest(req)
.build();
- AllocateBlockResponse resp = submitRequest(omRequest)
+ AllocateBlockResponse resp = handleError(submitRequest(omRequest))
.getAllocateBlockResponse();
-
- if (resp.getStatus() != OK) {
- throw new IOException("Allocate block failed, error:" +
- resp.getStatus());
- }
return OmKeyLocationInfo.getFromProtobuf(resp.getKeyLocation());
}
@@ -686,12 +639,9 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setCommitKeyRequest(req)
.build();
- CommitKeyResponse resp = submitRequest(omRequest).getCommitKeyResponse();
+ handleError(submitRequest(omRequest));
+
- if (resp.getStatus() != OK) {
- throw new IOException("Commit key failed, error:" +
- resp.getStatus());
- }
}
@@ -709,12 +659,9 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setLookupKeyRequest(req)
.build();
- LookupKeyResponse resp = submitRequest(omRequest).getLookupKeyResponse();
+ LookupKeyResponse resp =
+ handleError(submitRequest(omRequest)).getLookupKeyResponse();
- if (resp.getStatus() != OK) {
- throw new IOException("Lookup key failed, error:" +
- resp.getStatus());
- }
return OmKeyInfo.getFromProtobuf(resp.getKeyInfo());
}
@@ -733,12 +680,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setRenameKeyRequest(req)
.build();
- RenameKeyResponse resp = submitRequest(omRequest).getRenameKeyResponse();
-
- if (resp.getStatus() != OK) {
- throw new IOException("Rename key failed, error:" +
- resp.getStatus());
- }
+ handleError(submitRequest(omRequest));
}
/**
@@ -760,12 +702,8 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setDeleteKeyRequest(req)
.build();
- DeleteKeyResponse resp = submitRequest(omRequest).getDeleteKeyResponse();
+ handleError(submitRequest(omRequest));
- if (resp.getStatus() != OK) {
- throw new IOException("Delete key failed, error:" +
- resp.getStatus());
- }
}
/**
@@ -783,13 +721,8 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setDeleteBucketRequest(req)
.build();
- DeleteBucketResponse resp = submitRequest(omRequest)
- .getDeleteBucketResponse();
+ handleError(submitRequest(omRequest));
- if (resp.getStatus() != OK) {
- throw new
- IOException("Delete Bucket failed, error:" + resp.getStatus());
- }
}
/**
@@ -818,18 +751,14 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setListKeysRequest(req)
.build();
- ListKeysResponse resp = submitRequest(omRequest).getListKeysResponse();
+ ListKeysResponse resp =
+ handleError(submitRequest(omRequest)).getListKeysResponse();
+ keys.addAll(
+ resp.getKeyInfoList().stream()
+ .map(OmKeyInfo::getFromProtobuf)
+ .collect(Collectors.toList()));
+ return keys;
- if (resp.getStatus() == OK) {
- keys.addAll(
- resp.getKeyInfoList().stream()
- .map(OmKeyInfo::getFromProtobuf)
- .collect(Collectors.toList()));
- return keys;
- } else {
- throw new IOException("List Keys failed, error: "
- + resp.getStatus());
- }
}
@Override
@@ -844,13 +773,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setCreateS3BucketRequest(req)
.build();
- S3CreateBucketResponse resp = submitRequest(omRequest)
- .getCreateS3BucketResponse();
-
- if(resp.getStatus() != OK) {
- throw new IOException("Creating S3 bucket failed, error: "
- + resp.getStatus());
- }
+ handleError(submitRequest(omRequest));
}
@@ -864,13 +787,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setDeleteS3BucketRequest(request)
.build();
- S3DeleteBucketResponse resp = submitRequest(omRequest)
- .getDeleteS3BucketResponse();
-
- if(resp.getStatus() != OK) {
- throw new IOException("Creating S3 bucket failed, error: "
- + resp.getStatus());
- }
+ handleError(submitRequest(omRequest));
}
@@ -885,13 +802,8 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setInfoS3BucketRequest(request)
.build();
- S3BucketInfoResponse resp = submitRequest(omRequest)
+ S3BucketInfoResponse resp = handleError(submitRequest(omRequest))
.getInfoS3BucketResponse();
-
- if(resp.getStatus() != OK) {
- throw new IOException("GetOzoneBucketMapping failed, error:" + resp
- .getStatus());
- }
return resp.getOzoneMapping();
}
@@ -915,19 +827,15 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setListS3BucketsRequest(request)
.build();
- S3ListBucketsResponse resp = submitRequest(omRequest)
+ S3ListBucketsResponse resp = handleError(submitRequest(omRequest))
.getListS3BucketsResponse();
- if (resp.getStatus() == OK) {
- buckets.addAll(
- resp.getBucketInfoList().stream()
- .map(OmBucketInfo::getFromProtobuf)
- .collect(Collectors.toList()));
- return buckets;
- } else {
- throw new IOException("List S3 Buckets failed, error: "
- + resp.getStatus());
- }
+ buckets.addAll(
+ resp.getBucketInfoList().stream()
+ .map(OmBucketInfo::getFromProtobuf)
+ .collect(Collectors.toList()));
+ return buckets;
+
}
@Override
@@ -938,15 +846,11 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
OMRequest omRequest = createOMRequest(Type.GetS3Secret)
.setGetS3SecretRequest(request)
.build();
- final GetS3SecretResponse resp = submitRequest(omRequest)
+ final GetS3SecretResponse resp = handleError(submitRequest(omRequest))
.getGetS3SecretResponse();
- if(resp.getStatus() != OK) {
- throw new IOException("Fetch S3 Secret failed, error: " +
- resp.getStatus());
- } else {
- return S3SecretValue.fromProtobuf(resp.getS3Secret());
- }
+ return S3SecretValue.fromProtobuf(resp.getS3Secret());
+
}
/**
@@ -974,13 +878,9 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setInitiateMultiPartUploadRequest(multipartInfoInitiateRequest.build())
.build();
- MultipartInfoInitiateResponse resp = submitRequest(omRequest)
+ MultipartInfoInitiateResponse resp = handleError(submitRequest(omRequest))
.getInitiateMultiPartUploadResponse();
- if (resp.getStatus() != OK) {
- throw new IOException("Initiate Multipart upload failed, error:" + resp
- .getStatus());
- }
return new OmMultipartInfo(resp.getVolumeName(), resp.getBucketName(), resp
.getKeyName(), resp.getMultipartUploadID());
}
@@ -1016,14 +916,10 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.build())
.build();
- MultipartCommitUploadPartResponse response = submitRequest(omRequest)
+ MultipartCommitUploadPartResponse response =
+ handleError(submitRequest(omRequest))
.getCommitMultiPartUploadResponse();
- if (response.getStatus() != OK) {
- throw new IOException("Commit multipart upload part key failed, error:"
- + response.getStatus());
- }
-
OmMultipartCommitUploadPartInfo info = new
OmMultipartCommitUploadPartInfo(response.getPartName());
return info;
@@ -1051,14 +947,10 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setCompleteMultiPartUploadRequest(
multipartUploadCompleteRequest.build()).build();
- MultipartUploadCompleteResponse response = submitRequest(omRequest)
+ MultipartUploadCompleteResponse response =
+ handleError(submitRequest(omRequest))
.getCompleteMultiPartUploadResponse();
- if (response.getStatus() != OK) {
- throw new IOException("Complete multipart upload failed, error:" +
- response.getStatus());
- }
-
OmMultipartUploadCompleteInfo info = new
OmMultipartUploadCompleteInfo(response.getVolume(), response
.getBucket(), response.getKey(), response.getHash());
@@ -1082,13 +974,7 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setAbortMultiPartUploadRequest(multipartUploadAbortRequest.build())
.build();
- MultipartUploadAbortResponse response =
- submitRequest(omRequest).getAbortMultiPartUploadResponse();
-
- if (response.getStatus() != OK) {
- throw new IOException("Abort multipart upload failed, error:" +
- response.getStatus());
- }
+ handleError(submitRequest(omRequest));
}
@@ -1107,12 +993,9 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
multipartUploadListPartsRequest.build()).build();
MultipartUploadListPartsResponse response =
- submitRequest(omRequest).getListMultipartUploadPartsResponse();
+ handleError(submitRequest(omRequest))
+ .getListMultipartUploadPartsResponse();
- if (response.getStatus() != Status.OK) {
- throw new IOException("List Multipart upload parts failed, error: " +
- response.getStatus());
- }
OmMultipartUploadListParts omMultipartUploadListParts =
new OmMultipartUploadListParts(response.getType(),
@@ -1130,17 +1013,13 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
.setServiceListRequest(req)
.build();
- final ServiceListResponse resp = submitRequest(omRequest)
+ final ServiceListResponse resp = handleError(submitRequest(omRequest))
.getServiceListResponse();
- if (resp.getStatus() == OK) {
- return resp.getServiceInfoList().stream()
+ return resp.getServiceInfoList().stream()
.map(ServiceInfo::getFromProtobuf)
.collect(Collectors.toList());
- } else {
- throw new IOException("Getting service list failed, error: "
- + resp.getStatus());
- }
+
}
/**
@@ -1164,15 +1043,11 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
final GetDelegationTokenResponseProto resp;
try {
- resp = submitRequest(omRequest).getGetDelegationTokenResponse();
-
- if (resp.getStatus() == OK) {
- return resp.getResponse().hasToken() ?
- OMPBHelper.convertToDelegationToken(resp.getResponse().getToken())
- : null;
- }
- throw new OMException("Get delegation token failed with response:"
- + resp.getStatus(), toResultStatus(resp.getStatus()));
+ resp =
+ handleError(submitRequest(omRequest)).getGetDelegationTokenResponse();
+ return resp.getResponse().hasToken() ?
+ OMPBHelper.convertToDelegationToken(resp.getResponse().getToken())
+ : null;
} catch (IOException e) {
if(e instanceof OMException) {
throw (OMException)e;
@@ -1202,13 +1077,9 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
final RenewDelegationTokenResponseProto resp;
try {
- resp = submitRequest(omRequest)
+ resp = handleError(submitRequest(omRequest))
.getRenewDelegationTokenResponse();
- if (resp.getStatus() == OK) {
- return resp.getResponse().getNewExpiryTime();
- }
- throw new OMException("Renew delegation token failed with response:"
- + resp.getStatus(), toResultStatus(resp.getStatus()));
+ return resp.getResponse().getNewExpiryTime();
} catch (IOException e) {
if(e instanceof OMException) {
throw (OMException)e;
@@ -1237,12 +1108,9 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
final CancelDelegationTokenResponseProto resp;
try {
- resp = submitRequest(omRequest).getCancelDelegationTokenResponse();
- if (resp.getStatus() == OK) {
- return;
- }
- throw new OMException("Cancel delegation token failed with response:"
- + resp.getStatus(), toResultStatus(resp.getStatus()));
+ resp = handleError(submitRequest(omRequest))
+ .getCancelDelegationTokenResponse();
+ return;
} catch (IOException e) {
if(e instanceof OMException) {
throw (OMException)e;
@@ -1252,23 +1120,4 @@ public final class OzoneManagerProtocolClientSideTranslatorPB
}
}
- /**
- * Converts proto status to OMException result code.
- *
- * @param status Proto status received from rpc call.
- */
- public ResultCodes toResultStatus(Status status) {
- switch (status) {
- case INVALID_AUTH_METHOD:
- return ResultCodes.INVALID_AUTH_METHOD;
- case INVALID_TOKEN:
- return ResultCodes.INVALID_TOKEN;
- case TOKEN_EXPIRED:
- return ResultCodes.TOKEN_EXPIRED;
- case TOKEN_ERROR_OTHER:
- return TOKEN_ERROR_OTHER;
- default:
- return UNKNOWN;
- }
- }
}
diff --git a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
index 9fa4dfd..12ec29b 100644
--- a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
+++ b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
@@ -135,8 +135,11 @@ message OMResponse {
optional string traceID = 2;
optional bool success = 3 [default=true];
+
optional string message = 4;
+ required Status status = 5;
+
optional CreateVolumeResponse createVolumeResponse = 11;
optional SetVolumePropertyResponse setVolumePropertyResponse = 12;
optional CheckVolumeAccessResponse checkVolumeAccessResponse = 13;
@@ -217,6 +220,8 @@ enum Status {
TOKEN_EXPIRED = 35;
TOKEN_ERROR_OTHER = 36;
LIST_MULTIPART_UPLOAD_PARTS_FAILED = 37;
+ SCM_IN_CHILL_MODE = 38;
+ INVALID_REQUEST = 39;
}
@@ -239,7 +244,7 @@ message CreateVolumeRequest {
}
message CreateVolumeResponse {
- required Status status = 1;
+
}
message VolumeList {
@@ -256,7 +261,7 @@ message SetVolumePropertyRequest {
}
message SetVolumePropertyResponse {
- required Status status = 1;
+
}
/**
@@ -269,7 +274,6 @@ message CheckVolumeAccessRequest {
message CheckVolumeAccessResponse {
- required Status status = 1;
}
@@ -282,7 +286,6 @@ message InfoVolumeRequest {
}
message InfoVolumeResponse {
- required Status status = 1;
optional VolumeInfo volumeInfo = 2;
}
@@ -294,7 +297,7 @@ message DeleteVolumeRequest {
}
message DeleteVolumeResponse {
- required Status status = 1;
+
}
@@ -316,7 +319,6 @@ message ListVolumeRequest {
}
message ListVolumeResponse {
- required Status status = 1;
repeated VolumeInfo volumeInfo = 2;
}
@@ -369,7 +371,7 @@ message CreateBucketRequest {
}
message CreateBucketResponse {
- required Status status = 1;
+
}
message InfoBucketRequest {
@@ -378,7 +380,7 @@ message InfoBucketRequest {
}
message InfoBucketResponse {
- required Status status = 1;
+
optional BucketInfo bucketInfo = 2;
}
@@ -387,7 +389,7 @@ message SetBucketPropertyRequest {
}
message SetBucketPropertyResponse {
- required Status status = 1;
+
}
message DeleteBucketRequest {
@@ -396,7 +398,7 @@ message DeleteBucketRequest {
}
message DeleteBucketResponse {
- required Status status = 1;
+
}
message ListBucketsRequest {
@@ -407,7 +409,7 @@ message ListBucketsRequest {
}
message ListBucketsResponse {
- required Status status = 1;
+
repeated BucketInfo bucketInfo = 2;
}
@@ -459,7 +461,7 @@ message CreateKeyRequest {
}
message CreateKeyResponse {
- required Status status = 1;
+
optional KeyInfo keyInfo = 2;
// clients' followup request may carry this ID for stateful operations
// (similar to a cookie).
@@ -472,7 +474,7 @@ message LookupKeyRequest {
}
message LookupKeyResponse {
- required Status status = 1;
+
optional KeyInfo keyInfo = 2;
// clients' followup request may carry this ID for stateful operations (similar
// to a cookie).
@@ -487,7 +489,7 @@ message RenameKeyRequest{
}
message RenameKeyResponse{
- required Status status = 1;
+
}
message DeleteKeyRequest {
@@ -495,7 +497,7 @@ message DeleteKeyRequest {
}
message DeleteKeyResponse {
- required Status status = 1;
+
optional KeyInfo keyInfo = 2;
// clients' followup request may carry this ID for stateful operations
// (similar to a cookie).
@@ -531,7 +533,7 @@ message ListKeysRequest {
}
message ListKeysResponse {
- required Status status = 1;
+
repeated KeyInfo keyInfo = 2;
}
@@ -541,7 +543,7 @@ message CommitKeyRequest {
}
message CommitKeyResponse {
- required Status status = 1;
+
}
message AllocateBlockRequest {
@@ -550,7 +552,7 @@ message AllocateBlockRequest {
}
message AllocateBlockResponse {
- required Status status = 1;
+
optional KeyLocation keyLocation = 2;
}
@@ -558,7 +560,7 @@ message ServiceListRequest {
}
message ServiceListResponse {
- required Status status = 1;
+
repeated ServiceInfo serviceInfo = 2;
}
@@ -585,7 +587,7 @@ message S3CreateBucketRequest {
}
message S3CreateBucketResponse {
- required Status status = 1;
+
}
message S3DeleteBucketRequest {
@@ -593,14 +595,14 @@ message S3DeleteBucketRequest {
}
message S3DeleteBucketResponse {
- required Status status = 1;
+
}
message S3BucketInfoRequest {
required string s3bucketName = 1;
}
message S3BucketInfoResponse {
- required Status status = 1;
+
optional string ozoneMapping = 2;
}
@@ -612,7 +614,7 @@ message S3ListBucketsRequest {
}
message S3ListBucketsResponse {
- required Status status = 1;
+
repeated BucketInfo bucketInfo = 2;
}
@@ -625,7 +627,6 @@ message MultipartInfoInitiateResponse {
required string bucketName = 2;
required string keyName = 3;
required string multipartUploadID = 4;
- required Status status = 5;
}
message MultipartKeyInfo {
@@ -647,7 +648,6 @@ message MultipartCommitUploadPartRequest {
message MultipartCommitUploadPartResponse {
// This one is returned as Etag for S3.
optional string partName = 1;
- required Status status = 2;
}
message MultipartUploadCompleteRequest {
@@ -660,7 +660,6 @@ message MultipartUploadCompleteResponse {
optional string bucket = 2;
optional string key = 3;
optional string hash = 4; // This will be used as etag for s3
- required Status status = 5;
}
message Part {
@@ -673,7 +672,7 @@ message MultipartUploadAbortRequest {
}
message MultipartUploadAbortResponse {
- required Status status = 1;
+
}
message MultipartUploadListPartsRequest {
required string volume = 1;
@@ -685,7 +684,7 @@ message MultipartUploadListPartsRequest {
}
message MultipartUploadListPartsResponse {
- required Status status = 1;
+
optional hadoop.hdds.ReplicationType type = 2;
optional uint32 nextPartNumberMarker = 3;
optional bool isTruncated = 4;
@@ -701,17 +700,17 @@ message PartInfo {
}
message GetDelegationTokenResponseProto{
- required Status status = 1;
+
optional hadoop.common.GetDelegationTokenResponseProto response = 2;
}
message RenewDelegationTokenResponseProto{
- required Status status = 1;
+
optional hadoop.common.RenewDelegationTokenResponseProto response = 2;
}
message CancelDelegationTokenResponseProto {
- required Status status = 1;
+
optional hadoop.common.CancelDelegationTokenResponseProto response = 2;
}
@@ -725,7 +724,6 @@ message GetS3SecretRequest {
}
message GetS3SecretResponse {
- required Status status = 1;
required S3Secret s3Secret = 2;
}
diff --git a/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/exceptions/TestResultCodes.java b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/exceptions/TestResultCodes.java
new file mode 100644
index 0000000..24b5307
--- /dev/null
+++ b/hadoop-ozone/common/src/test/java/org/apache/hadoop/ozone/om/exceptions/TestResultCodes.java
@@ -0,0 +1,49 @@
+/**
+ * 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.exceptions;
+
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test code mappping.
+ */
+public class TestResultCodes {
+
+ @Test
+ public void codeMapping() {
+ Assert.assertEquals(ResultCodes.values().length, Status.values().length);
+ for (int i = 0; i < ResultCodes.values().length; i++) {
+ ResultCodes codeValue = ResultCodes.values()[i];
+ Status protoBufValue = Status.values()[i];
+ Assert.assertTrue(String
+ .format("Protobuf/Enum constant name mismatch %s %s", codeValue,
+ protoBufValue), sameName(codeValue.name(), protoBufValue.name()));
+ ResultCodes converted = ResultCodes.values()[protoBufValue.ordinal()];
+ Assert.assertEquals(codeValue, converted);
+
+ }
+ }
+
+ private boolean sameName(String codeValue, String protoBufValue) {
+ return codeValue.equals(protoBufValue);
+ }
+}
\ No newline at end of file
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java
index 7ea259d..2023e0e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/OzoneTestUtils.java
@@ -17,18 +17,21 @@
*/
package org.apache.hadoop.ozone;
+import java.io.IOException;
+import java.util.List;
+
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.test.LambdaTestUtils.VoidCallable;
+
import org.apache.ratis.util.function.CheckedConsumer;
import org.junit.Assert;
-import java.io.IOException;
-import java.util.List;
-
/**
* Helper class for Tests.
*/
@@ -95,4 +98,15 @@ public final class OzoneTestUtils {
}
}
+ public static <E extends Throwable> void expectOmException(
+ OMException.ResultCodes code,
+ VoidCallable eval)
+ throws Exception {
+ try {
+ eval.call();
+ Assert.fail("OMException is expected");
+ } catch (OMException ex) {
+ Assert.assertEquals(code, ex.getResult());
+ }
+ }
}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java
index 29264c0..2a4dbd5 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestSecureOzoneCluster.java
@@ -24,6 +24,7 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_AUTH_METHOD;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKEN_ERROR_OTHER;
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKEN_EXPIRED;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
import static org.slf4j.event.Level.INFO;
import java.io.File;
@@ -415,8 +416,8 @@ public final class TestSecureOzoneCluster {
// Case 3: Test Client can authenticate using token.
Assert.assertFalse(logs.getOutput().contains(
"Auth successful for " + username + " (auth:TOKEN)"));
- LambdaTestUtils.intercept(IOException.class, "Delete Volume failed,"
- + " error:VOLUME_NOT_FOUND", () -> omClient.deleteVolume("vol1"));
+ OzoneTestUtils.expectOmException(VOLUME_NOT_FOUND,
+ () -> omClient.deleteVolume("vol1"));
Assert.assertTrue(logs.getOutput().contains("Auth successful for "
+ username + " (auth:TOKEN)"));
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
deleted file mode 100644
index a7debef..0000000
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/TestOzoneRestClient.java
+++ /dev/null
@@ -1,499 +0,0 @@
-/**
- * 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.client.rest;
-
-import org.apache.commons.lang3.RandomStringUtils;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
-import org.apache.hadoop.hdds.protocol.StorageType;
-import org.apache.hadoop.hdds.protocol.DatanodeDetails;
-import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
-import org.apache.hadoop.hdds.scm.container.ContainerID;
-import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
-import org.apache.hadoop.ozone.*;
-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.io.OzoneInputStream;
-import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
-import org.apache.hadoop.ozone.container.common.helpers.BlockData;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueBlockIterator;
-import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.UUID;
-
-/**
- * This class is to test all the public facing APIs of Ozone REST Client.
- */
-public class TestOzoneRestClient {
-
- @Rule
- public ExpectedException thrown = ExpectedException.none();
-
- private static MiniOzoneCluster cluster = null;
- private static OzoneClient ozClient = null;
- private static ObjectStore store = null;
-
- /**
- * Create a MiniDFSCluster for testing.
- * <p>
- * Ozone is made active by setting OZONE_ENABLED = true
- *
- * @throws IOException
- */
- @BeforeClass
- public static void init() throws Exception {
- OzoneConfiguration conf = new OzoneConfiguration();
- cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
- cluster.waitForClusterToBeReady();
- InetSocketAddress omHttpAddress = cluster.getOzoneManager()
- .getHttpServer().getHttpAddress();
- ozClient = OzoneClientFactory.getRestClient(omHttpAddress.getHostName(),
- omHttpAddress.getPort(), conf);
- store = ozClient.getObjectStore();
- }
-
-
- @Test
- public void testCreateVolume()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- Assert.assertEquals(volumeName, volume.getName());
- }
-
- @Test
- public void testCreateVolumeWithOwner()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- VolumeArgs.Builder argsBuilder = VolumeArgs.newBuilder();
- argsBuilder.setOwner("test");
- store.createVolume(volumeName, argsBuilder.build());
- OzoneVolume volume = store.getVolume(volumeName);
- Assert.assertEquals(volumeName, volume.getName());
- Assert.assertEquals("test", volume.getOwner());
- }
-
- @Test
- public void testCreateVolumeWithQuota()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- VolumeArgs.Builder argsBuilder = VolumeArgs.newBuilder();
- argsBuilder.setOwner("test").setQuota("1000000000 BYTES");
- store.createVolume(volumeName, argsBuilder.build());
- OzoneVolume volume = store.getVolume(volumeName);
- Assert.assertEquals(volumeName, volume.getName());
- Assert.assertEquals("test", volume.getOwner());
- Assert.assertEquals(1000000000L, volume.getQuota());
- }
-
- @Test
- public void testVolumeAlreadyExist()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- try {
- store.createVolume(volumeName);
- } catch (IOException ex) {
- Assert.assertEquals(
- "Volume creation failed, error:VOLUME_ALREADY_EXISTS",
- ex.getCause().getMessage());
- }
- }
-
- @Test
- public void testSetVolumeOwner()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- store.getVolume(volumeName).setOwner("test");
- OzoneVolume volume = store.getVolume(volumeName);
- Assert.assertEquals("test", volume.getOwner());
- }
-
- @Test
- public void testSetVolumeQuota()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- store.getVolume(volumeName).setQuota(
- OzoneQuota.parseQuota("100000000 BYTES"));
- OzoneVolume volume = store.getVolume(volumeName);
- Assert.assertEquals(100000000L, volume.getQuota());
- }
-
- @Test
- public void testDeleteVolume()
- throws IOException, OzoneException {
- thrown.expectMessage("Info Volume failed, error");
- String volumeName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- Assert.assertNotNull(volume);
- store.deleteVolume(volumeName);
- store.getVolume(volumeName);
- }
-
- @Test
- public void testCreateBucket()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
- Assert.assertEquals(bucketName, bucket.getName());
- }
-
- @Test
- public void testCreateBucketWithVersioning()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- BucketArgs.Builder builder = BucketArgs.newBuilder();
- builder.setVersioning(true);
- volume.createBucket(bucketName, builder.build());
- OzoneBucket bucket = volume.getBucket(bucketName);
- Assert.assertEquals(bucketName, bucket.getName());
- Assert.assertEquals(true, bucket.getVersioning());
- }
-
- @Test
- public void testCreateBucketWithStorageType()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- BucketArgs.Builder builder = BucketArgs.newBuilder();
- builder.setStorageType(StorageType.SSD);
- volume.createBucket(bucketName, builder.build());
- OzoneBucket bucket = volume.getBucket(bucketName);
- Assert.assertEquals(bucketName, bucket.getName());
- Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
- }
-
- @Test
- public void testCreateBucketWithAcls()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test",
- OzoneAcl.OzoneACLRights.READ_WRITE);
- List<OzoneAcl> acls = new ArrayList<>();
- acls.add(userAcl);
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- BucketArgs.Builder builder = BucketArgs.newBuilder();
- builder.setAcls(acls);
- volume.createBucket(bucketName, builder.build());
- OzoneBucket bucket = volume.getBucket(bucketName);
- Assert.assertEquals(bucketName, bucket.getName());
- Assert.assertTrue(bucket.getAcls().contains(userAcl));
- }
-
- @Test
- public void testCreateBucketWithAllArgument()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test",
- OzoneAcl.OzoneACLRights.READ_WRITE);
- List<OzoneAcl> acls = new ArrayList<>();
- acls.add(userAcl);
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- BucketArgs.Builder builder = BucketArgs.newBuilder();
- builder.setVersioning(true)
- .setStorageType(StorageType.SSD)
- .setAcls(acls);
- volume.createBucket(bucketName, builder.build());
- OzoneBucket bucket = volume.getBucket(bucketName);
- Assert.assertEquals(bucketName, bucket.getName());
- Assert.assertEquals(true, bucket.getVersioning());
- Assert.assertEquals(StorageType.SSD, bucket.getStorageType());
- Assert.assertTrue(bucket.getAcls().contains(userAcl));
- }
-
- @Test
- public void testAddBucketAcl()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- List<OzoneAcl> acls = new ArrayList<>();
- acls.add(new OzoneAcl(
- OzoneAcl.OzoneACLType.USER, "test",
- OzoneAcl.OzoneACLRights.READ_WRITE));
- OzoneBucket bucket = volume.getBucket(bucketName);
- bucket.addAcls(acls);
- OzoneBucket newBucket = volume.getBucket(bucketName);
- Assert.assertEquals(bucketName, newBucket.getName());
- Assert.assertTrue(bucket.getAcls().contains(acls.get(0)));
- }
-
- @Test
- public void testRemoveBucketAcl()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- OzoneAcl userAcl = new OzoneAcl(OzoneAcl.OzoneACLType.USER, "test",
- OzoneAcl.OzoneACLRights.READ_WRITE);
- List<OzoneAcl> acls = new ArrayList<>();
- acls.add(userAcl);
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- BucketArgs.Builder builder = BucketArgs.newBuilder();
- builder.setAcls(acls);
- volume.createBucket(bucketName, builder.build());
- OzoneBucket bucket = volume.getBucket(bucketName);
- bucket.removeAcls(acls);
- OzoneBucket newBucket = volume.getBucket(bucketName);
- Assert.assertEquals(bucketName, newBucket.getName());
- Assert.assertTrue(!bucket.getAcls().contains(acls.get(0)));
- }
-
- @Test
- public void testSetBucketVersioning()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
- bucket.setVersioning(true);
- OzoneBucket newBucket = volume.getBucket(bucketName);
- Assert.assertEquals(bucketName, newBucket.getName());
- Assert.assertEquals(true, newBucket.getVersioning());
- }
-
- @Test
- public void testSetBucketStorageType()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
- bucket.setStorageType(StorageType.SSD);
- OzoneBucket newBucket = volume.getBucket(bucketName);
- Assert.assertEquals(bucketName, newBucket.getName());
- Assert.assertEquals(StorageType.SSD, newBucket.getStorageType());
- }
-
-
- @Test
- public void testDeleteBucket()
- throws IOException, OzoneException {
- thrown.expectMessage("Info Bucket failed, error");
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
- Assert.assertNotNull(bucket);
- volume.deleteBucket(bucketName);
- volume.getBucket(bucketName);
- }
-
-
- @Test
- public void testPutKey()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
-
- String value = "sample value";
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
-
- for (int i = 0; i < 10; i++) {
- String keyName = UUID.randomUUID().toString();
-
- OzoneOutputStream out = bucket.createKey(keyName,
- value.getBytes().length, ReplicationType.STAND_ALONE,
- ReplicationFactor.ONE, new HashMap<>());
- out.write(value.getBytes());
- out.close();
- OzoneKey key = bucket.getKey(keyName);
- Assert.assertEquals(keyName, key.getName());
- OzoneInputStream is = bucket.readKey(keyName);
- byte[] fileContent = new byte[value.getBytes().length];
- is.read(fileContent);
- Assert.assertEquals(value, new String(fileContent));
- }
- }
-
- @Test
- public void testDeleteKey()
- throws IOException, OzoneException {
- thrown.expectMessage("Lookup key failed, error");
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- String keyName = UUID.randomUUID().toString();
- String value = "sample value";
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
- OzoneOutputStream out = bucket.createKey(keyName,
- value.getBytes().length, ReplicationType.STAND_ALONE,
- ReplicationFactor.ONE, new HashMap<>());
- out.write(value.getBytes());
- out.close();
- OzoneKey key = bucket.getKey(keyName);
- Assert.assertEquals(keyName, key.getName());
- bucket.deleteKey(keyName);
- bucket.getKey(keyName);
- }
-
- @Test
- public void testRenameKey()
- throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- String fromKeyName = UUID.randomUUID().toString();
- String value = "sample value";
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
- OzoneOutputStream out = bucket.createKey(fromKeyName,
- value.getBytes().length, ReplicationType.STAND_ALONE,
- ReplicationFactor.ONE, new HashMap<>());
- out.write(value.getBytes());
- out.close();
- OzoneKey key = bucket.getKey(fromKeyName);
- Assert.assertEquals(fromKeyName, key.getName());
-
- String toKeyName = UUID.randomUUID().toString();
- bucket.renameKey(fromKeyName, toKeyName);
-
- key = bucket.getKey(toKeyName);
- Assert.assertEquals(toKeyName, key.getName());
-
- // Lookup for old key should fail.
- thrown.expectMessage("Lookup key failed, error");
- bucket.getKey(fromKeyName);
- }
-
- @Test
- public void testGetKeyDetails() throws IOException, OzoneException {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
- String keyName = UUID.randomUUID().toString();
- String keyValue = RandomStringUtils.random(128);
- OzoneOutputStream out = bucket.createKey(keyName,
- keyValue.getBytes().length, ReplicationType.STAND_ALONE,
- ReplicationFactor.ONE, new HashMap<>());
- out.write(keyValue.getBytes());
- out.close();
-
- // Get the containerID and localID.
- OzoneKeyDetails keyDetails = (OzoneKeyDetails)bucket.getKey(keyName);
- Assert.assertEquals(keyName, keyDetails.getName());
- List<OzoneKeyLocation> keyLocations = keyDetails.getOzoneKeyLocations();
- Assert.assertEquals(1, keyLocations.size());
- Long containerID = keyLocations.get(0).getContainerID();
- Long localID = keyLocations.get(0).getLocalID();
-
- // Make sure that the data size matched.
- Assert.assertEquals(keyValue.getBytes().length,
- keyLocations.get(0).getLength());
-
- // Sum the data size from chunks in Container via containerID
- // and localID, make sure the size equals to the actually value size.
- Pipeline pipeline = cluster.getStorageContainerManager()
- .getPipelineManager().getPipeline(
- cluster.getStorageContainerManager()
- .getContainerManager().getContainer(
- ContainerID.valueof(containerID)).getPipelineID());
- List<DatanodeDetails> datanodes = pipeline.getNodes();
- Assert.assertEquals(datanodes.size(), 1);
-
- DatanodeDetails datanodeDetails = datanodes.get(0);
- Assert.assertNotNull(datanodeDetails);
- HddsDatanodeService datanodeService = null;
- for (HddsDatanodeService datanodeServiceItr : cluster.getHddsDatanodes()) {
- if (datanodeDetails.equals(datanodeServiceItr.getDatanodeDetails())) {
- datanodeService = datanodeServiceItr;
- break;
- }
- }
- KeyValueContainerData containerData =
- (KeyValueContainerData)(datanodeService.getDatanodeStateMachine()
- .getContainer().getContainerSet().getContainer(containerID)
- .getContainerData());
- String containerPath = new File(containerData.getMetadataPath())
- .getParent();
- KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
- containerID, new File(containerPath));
- long valueLength = 0;
- while (keyValueBlockIterator.hasNext()) {
- BlockData blockData = keyValueBlockIterator.nextBlock();
- if (blockData.getBlockID().getLocalID() == localID) {
- List<ContainerProtos.ChunkInfo> chunks = blockData.getChunks();
- for (ContainerProtos.ChunkInfo chunk : chunks) {
- valueLength += chunk.getLen();
- }
- }
- }
- Assert.assertEquals(keyValue.getBytes().length, valueLength);
- }
-
- /**
- * Close OzoneClient and shutdown MiniDFSCluster.
- */
- @AfterClass
- public static void shutdown() throws IOException {
- if(ozClient != null) {
- ozClient.close();
- }
- if (cluster != null) {
- cluster.shutdown();
- }
- }
-}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/package-info.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/package-info.java
deleted file mode 100644
index c8940e4..0000000
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rest/package-info.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/**
- * 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.rest;
-
-/**
- * This package contains test class for Ozone rest client library.
- */
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
index f5334dd..fe04b56 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneRpcClientAbstract.java
@@ -39,19 +39,16 @@ import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.XceiverClientRatis;
-import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
-import org.apache.hadoop.hdds.scm.protocolPB
- .StorageContainerLocationProtocolClientSideTranslatorPB;
+import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConsts;
+import org.apache.hadoop.ozone.OzoneTestUtils;
import org.apache.hadoop.ozone.client.BucketArgs;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneBucket;
@@ -63,7 +60,6 @@ import org.apache.hadoop.ozone.client.OzoneKeyLocation;
import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts;
import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.VolumeArgs;
-import org.apache.hadoop.ozone.client.io.KeyOutputStream;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.client.rest.OzoneException;
@@ -74,6 +70,8 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueBlockIterator;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyValueContainerLocationUtil;
import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
@@ -81,19 +79,19 @@ import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
import org.apache.hadoop.util.Time;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
+import static java.nio.charset.StandardCharsets.UTF_8;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.either;
+import org.junit.Assert;
import static org.junit.Assert.assertNotEquals;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.fail;
+import org.junit.Ignore;
+import org.junit.Test;
/**
* This is an abstract class to test all the public facing APIs of Ozone
@@ -103,8 +101,6 @@ import static org.junit.Assert.fail;
* tests the Ozone Client by submitting requests to OM's Ratis server.
*/
public abstract class TestOzoneRpcClientAbstract {
- @Rule
- public ExpectedException thrown = ExpectedException.none();
private static MiniOzoneCluster cluster = null;
private static OzoneClient ozClient = null;
@@ -113,7 +109,7 @@ public abstract class TestOzoneRpcClientAbstract {
private static StorageContainerLocationProtocolClientSideTranslatorPB
storageContainerLocationClient;
- private static String SCM_ID = UUID.randomUUID().toString();
+ private static String scmId = UUID.randomUUID().toString();
/**
* Create a MiniOzoneCluster for testing.
@@ -123,7 +119,7 @@ public abstract class TestOzoneRpcClientAbstract {
static void startCluster(OzoneConfiguration conf) throws Exception {
cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(10)
- .setScmId(SCM_ID)
+ .setScmId(scmId)
.build();
cluster.waitForClusterToBeReady();
ozClient = OzoneClientFactory.getRpcClient(conf);
@@ -178,7 +174,7 @@ public abstract class TestOzoneRpcClientAbstract {
}
public static void setScmId(String scmId){
- TestOzoneRpcClientAbstract.SCM_ID = scmId;
+ TestOzoneRpcClientAbstract.scmId = scmId;
}
@Test
@@ -194,14 +190,15 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testDeleteVolume()
- throws IOException, OzoneException {
- thrown.expectMessage("Info Volume failed, error");
+ throws Exception {
String volumeName = UUID.randomUUID().toString();
store.createVolume(volumeName);
OzoneVolume volume = store.getVolume(volumeName);
Assert.assertNotNull(volume);
store.deleteVolume(volumeName);
- store.getVolume(volumeName);
+ OzoneTestUtils.expectOmException(ResultCodes.VOLUME_NOT_FOUND,
+ () -> store.getVolume(volumeName));
+
}
@Test
@@ -299,7 +296,7 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testDeleteS3Bucket()
- throws IOException, OzoneException {
+ throws Exception {
long currentTime = Time.now();
String userName = "ozone1";
String bucketName = UUID.randomUUID().toString();
@@ -311,8 +308,9 @@ public abstract class TestOzoneRpcClientAbstract {
Assert.assertTrue(bucket.getCreationTime() >= currentTime);
Assert.assertTrue(volume.getCreationTime() >= currentTime);
store.deleteS3Bucket(bucketName);
- thrown.expect(IOException.class);
- store.getOzoneVolumeName(bucketName);
+
+ OzoneTestUtils.expectOmException(ResultCodes.S3_BUCKET_NOT_FOUND,
+ () -> store.getOzoneVolumeName(bucketName));
}
@Test
@@ -416,14 +414,17 @@ public abstract class TestOzoneRpcClientAbstract {
}
@Test
- public void testInvalidBucketCreation() throws IOException {
- thrown.expectMessage("Bucket or Volume name has an unsupported" +
- " character : #");
+ public void testInvalidBucketCreation() throws Exception {
+
String volumeName = UUID.randomUUID().toString();
String bucketName = "invalid#bucket";
store.createVolume(volumeName);
OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
+ LambdaTestUtils.intercept(IllegalArgumentException.class,
+ "Bucket or Volume name has an unsupported" +
+ " character : #",
+ () -> volume.createBucket(bucketName));
+
}
@Test
@@ -499,8 +500,8 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testDeleteBucket()
- throws IOException, OzoneException {
- thrown.expectMessage("Info Bucket failed, error");
+ throws Exception {
+
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
store.createVolume(volumeName);
@@ -509,7 +510,10 @@ public abstract class TestOzoneRpcClientAbstract {
OzoneBucket bucket = volume.getBucket(bucketName);
Assert.assertNotNull(bucket);
volume.deleteBucket(bucketName);
- volume.getBucket(bucketName);
+
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND,
+ () -> volume.getBucket(bucketName)
+ );
}
private boolean verifyRatisReplication(String volumeName, String bucketName,
@@ -935,7 +939,7 @@ public abstract class TestOzoneRpcClientAbstract {
String containreBaseDir =
container.getContainerData().getVolume().getHddsRootDir().getPath();
File chunksLocationPath = KeyValueContainerLocationUtil
- .getChunksLocationPath(containreBaseDir, SCM_ID, containerID);
+ .getChunksLocationPath(containreBaseDir, scmId, containerID);
File chunkFile = new File(chunksLocationPath, chunkName);
// Corrupt the contents of the chunk file
@@ -945,8 +949,8 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testDeleteKey()
- throws IOException, OzoneException {
- thrown.expectMessage("Lookup key failed, error");
+ throws Exception {
+
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String keyName = UUID.randomUUID().toString();
@@ -963,12 +967,14 @@ public abstract class TestOzoneRpcClientAbstract {
OzoneKey key = bucket.getKey(keyName);
Assert.assertEquals(keyName, key.getName());
bucket.deleteKey(keyName);
- bucket.getKey(keyName);
+
+ OzoneTestUtils.expectOmException(ResultCodes.KEY_NOT_FOUND,
+ () -> bucket.getKey(keyName));
}
@Test
public void testRenameKey()
- throws IOException, OzoneException {
+ throws IOException {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String fromKeyName = UUID.randomUUID().toString();
@@ -986,14 +992,14 @@ public abstract class TestOzoneRpcClientAbstract {
Assert.assertEquals(fromKeyName, key.getName());
// Rename to empty string should fail.
- IOException ioe = null;
+ OMException oe = null;
String toKeyName = "";
try {
bucket.renameKey(fromKeyName, toKeyName);
- } catch (IOException e) {
- ioe = e;
+ } catch (OMException e) {
+ oe = e;
}
- Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error"));
+ Assert.assertEquals(ResultCodes.INVALID_KEY_NAME, oe.getResult());
toKeyName = UUID.randomUUID().toString();
bucket.renameKey(fromKeyName, toKeyName);
@@ -1001,10 +1007,10 @@ public abstract class TestOzoneRpcClientAbstract {
// Lookup for old key should fail.
try {
bucket.getKey(fromKeyName);
- } catch (IOException e) {
- ioe = e;
+ } catch (OMException e) {
+ oe = e;
}
- Assert.assertTrue(ioe.getMessage().contains("Lookup key failed, error"));
+ Assert.assertEquals(ResultCodes.KEY_NOT_FOUND, oe.getResult());
key = bucket.getKey(toKeyName);
Assert.assertEquals(toKeyName, key.getName());
@@ -1014,7 +1020,7 @@ public abstract class TestOzoneRpcClientAbstract {
// TODO: fix this
@Ignore
@Test
- public void testListVolume() throws IOException, OzoneException {
+ public void testListVolume() throws IOException {
String volBase = "vol-" + RandomStringUtils.randomNumeric(3);
//Create 10 volume vol-<random>-a-0-<random> to vol-<random>-a-9-<random>
String volBaseNameA = volBase + "-a-";
@@ -1057,7 +1063,7 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testListBucket()
- throws IOException, OzoneException {
+ throws IOException {
String volumeA = "vol-a-" + RandomStringUtils.randomNumeric(5);
String volumeB = "vol-b-" + RandomStringUtils.randomNumeric(5);
store.createVolume(volumeA);
@@ -1133,7 +1139,7 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testListBucketsOnEmptyVolume()
- throws IOException, OzoneException {
+ throws IOException {
String volume = "vol-" + RandomStringUtils.randomNumeric(5);
store.createVolume(volume);
OzoneVolume vol = store.getVolume(volume);
@@ -1145,7 +1151,7 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testListKey()
- throws IOException, OzoneException {
+ throws IOException {
String volumeA = "vol-a-" + RandomStringUtils.randomNumeric(5);
String volumeB = "vol-b-" + RandomStringUtils.randomNumeric(5);
String bucketA = "buc-a-" + RandomStringUtils.randomNumeric(5);
@@ -1280,7 +1286,7 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testListKeyOnEmptyBucket()
- throws IOException, OzoneException {
+ throws IOException {
String volume = "vol-" + RandomStringUtils.randomNumeric(5);
String bucket = "buc-" + RandomStringUtils.randomNumeric(5);
store.createVolume(volume);
@@ -1509,7 +1515,7 @@ public abstract class TestOzoneRpcClientAbstract {
}
@Test
- public void testNoSuchUploadError() throws IOException {
+ public void testNoSuchUploadError() throws Exception {
String volumeName = UUID.randomUUID().toString();
String bucketName = UUID.randomUUID().toString();
String keyName = UUID.randomUUID().toString();
@@ -1521,13 +1527,10 @@ public abstract class TestOzoneRpcClientAbstract {
OzoneBucket bucket = volume.getBucket(bucketName);
String uploadID = "random";
- try {
- bucket.createMultipartKey(keyName, sampleData.length(), 1, uploadID);
- fail("testNoSuchUploadError failed");
- } catch (IOException ex) {
- GenericTestUtils.assertExceptionContains("NO_SUCH_MULTIPART_UPLOAD_ERROR",
- ex);
- }
+ OzoneTestUtils
+ .expectOmException(ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR, () ->
+ bucket
+ .createMultipartKey(keyName, sampleData.length(), 1, uploadID));
}
@Test
@@ -1585,27 +1588,20 @@ public abstract class TestOzoneRpcClientAbstract {
Map<Integer, String> partsMap = new TreeMap<>();
// Uploading part 1 with less than min size
String partName = uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(
- "UTF-8"));
+ UTF_8));
partsMap.put(1, partName);
partName = uploadPart(bucket, keyName, uploadID, 2, "data".getBytes(
- "UTF-8"));
+ UTF_8));
partsMap.put(2, partName);
// Complete multipart upload
- try {
- completeMultipartUpload(bucket, keyName, uploadID, partsMap);
- fail("testMultipartUploadWithPartsLessThanMinSize failed");
- } catch (IOException ex) {
- GenericTestUtils.assertExceptionContains("ENTITY_TOO_SMALL", ex);
- }
+ OzoneTestUtils.expectOmException(ResultCodes.ENTITY_TOO_SMALL,
+ () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
}
-
-
-
@Test
public void testMultipartUploadWithPartsMisMatchWithListSizeDifferent()
throws Exception {
@@ -1626,12 +1622,8 @@ public abstract class TestOzoneRpcClientAbstract {
TreeMap<Integer, String> partsMap = new TreeMap<>();
partsMap.put(1, UUID.randomUUID().toString());
- try {
- completeMultipartUpload(bucket, keyName, uploadID, partsMap);
- fail("testMultipartUploadWithPartsMisMatch");
- } catch (IOException ex) {
- GenericTestUtils.assertExceptionContains("MISMATCH_MULTIPART_LIST", ex);
- }
+ OzoneTestUtils.expectOmException(ResultCodes.MISMATCH_MULTIPART_LIST,
+ () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
}
@@ -1650,18 +1642,14 @@ public abstract class TestOzoneRpcClientAbstract {
String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType
.STAND_ALONE, ReplicationFactor.ONE);
- uploadPart(bucket, keyName, uploadID, 1, "data".getBytes("UTF-8"));
+ uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
// We have not uploaded any parts, but passing some list it should throw
// error.
TreeMap<Integer, String> partsMap = new TreeMap<>();
partsMap.put(1, UUID.randomUUID().toString());
- try {
- completeMultipartUpload(bucket, keyName, uploadID, partsMap);
- fail("testMultipartUploadWithPartsMisMatch");
- } catch (IOException ex) {
- GenericTestUtils.assertExceptionContains("MISMATCH_MULTIPART_LIST", ex);
- }
+ OzoneTestUtils.expectOmException(ResultCodes.MISMATCH_MULTIPART_LIST,
+ () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
}
@@ -1679,18 +1667,14 @@ public abstract class TestOzoneRpcClientAbstract {
String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType
.STAND_ALONE, ReplicationFactor.ONE);
- uploadPart(bucket, keyName, uploadID, 1, "data".getBytes("UTF-8"));
+ uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
// We have not uploaded any parts, but passing some list it should throw
// error.
TreeMap<Integer, String> partsMap = new TreeMap<>();
partsMap.put(3, "random");
- try {
- completeMultipartUpload(bucket, keyName, uploadID, partsMap);
- fail("testMultipartUploadWithPartsMisMatch");
- } catch (IOException ex) {
- GenericTestUtils.assertExceptionContains("MISSING_UPLOAD_PARTS", ex);
- }
+ OzoneTestUtils.expectOmException(ResultCodes.MISSING_UPLOAD_PARTS,
+ () -> completeMultipartUpload(bucket, keyName, uploadID, partsMap));
}
@Test
@@ -1704,13 +1688,8 @@ public abstract class TestOzoneRpcClientAbstract {
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
- try {
- bucket.abortMultipartUpload(keyName, "random");
- fail("testAbortUploadFail failed");
- } catch (IOException ex) {
- GenericTestUtils.assertExceptionContains(
- "NO_SUCH_MULTIPART_UPLOAD_ERROR", ex);
- }
+ OzoneTestUtils.expectOmException(ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR,
+ () -> bucket.abortMultipartUpload(keyName, "random"));
}
@@ -1725,13 +1704,9 @@ public abstract class TestOzoneRpcClientAbstract {
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
- try {
- String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType
- .STAND_ALONE, ReplicationFactor.ONE);
- bucket.abortMultipartUpload(keyName, uploadID);
- } catch (IOException ex) {
- fail("testAbortUploadSuccess failed");
- }
+ String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType
+ .STAND_ALONE, ReplicationFactor.ONE);
+ bucket.abortMultipartUpload(keyName, uploadID);
}
@Test
@@ -1745,14 +1720,10 @@ public abstract class TestOzoneRpcClientAbstract {
volume.createBucket(bucketName);
OzoneBucket bucket = volume.getBucket(bucketName);
- try {
- String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType
- .STAND_ALONE, ReplicationFactor.ONE);
- uploadPart(bucket, keyName, uploadID, 1, "data".getBytes("UTF-8"));
- bucket.abortMultipartUpload(keyName, uploadID);
- } catch (IOException ex) {
- fail("testAbortUploadSuccess failed");
- }
+ String uploadID = initiateMultipartUpload(bucket, keyName, ReplicationType
+ .STAND_ALONE, ReplicationFactor.ONE);
+ uploadPart(bucket, keyName, uploadID, 1, "data".getBytes(UTF_8));
+ bucket.abortMultipartUpload(keyName, uploadID);
}
@Test
@@ -1947,20 +1918,19 @@ public abstract class TestOzoneRpcClientAbstract {
@Test
public void testListPartsWithInvalidUploadID() throws Exception {
- try {
- String volumeName = UUID.randomUUID().toString();
- String bucketName = UUID.randomUUID().toString();
- String keyName = UUID.randomUUID().toString();
+ OzoneTestUtils
+ .expectOmException(ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR, () -> {
+ String volumeName = UUID.randomUUID().toString();
+ String bucketName = UUID.randomUUID().toString();
+ String keyName = UUID.randomUUID().toString();
- store.createVolume(volumeName);
- OzoneVolume volume = store.getVolume(volumeName);
- volume.createBucket(bucketName);
- OzoneBucket bucket = volume.getBucket(bucketName);
- OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
- bucket.listParts(keyName, "random", 100, 2);
- } catch (IOException ex) {
- GenericTestUtils.assertExceptionContains("NO_SUCH_MULTIPART_UPLOAD", ex);
- }
+ store.createVolume(volumeName);
+ OzoneVolume volume = store.getVolume(volumeName);
+ volume.createBucket(bucketName);
+ OzoneBucket bucket = volume.getBucket(bucketName);
+ OzoneMultipartUploadPartListParts ozoneMultipartUploadPartListParts =
+ bucket.listParts(keyName, "random", 100, 2);
+ });
}
@@ -1995,9 +1965,9 @@ public abstract class TestOzoneRpcClientAbstract {
String part3 = UUID.randomUUID().toString();
partName = uploadPart(bucket, keyName, uploadID, 3, part3.getBytes(
- "UTF-8"));
+ UTF_8));
partsMap.put(3, partName);
- length += part3.getBytes("UTF-8").length;
+ length += part3.getBytes(UTF_8).length;
// Complete multipart upload request
@@ -2006,7 +1976,7 @@ public abstract class TestOzoneRpcClientAbstract {
//Now Read the key which has been completed multipart upload.
byte[] fileContent = new byte[data.length + data.length + part3.getBytes(
- "UTF-8").length];
+ UTF_8).length];
OzoneInputStream inputStream = bucket.readKey(keyName);
inputStream.read(fileContent);
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmAcls.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmAcls.java
index addfd12..644460d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmAcls.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOmAcls.java
@@ -16,12 +16,15 @@
*/
package org.apache.hadoop.ozone.om;
+import java.util.LinkedList;
+import java.util.UUID;
-import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneTestUtils;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.IOzoneObj;
import org.apache.hadoop.ozone.security.acl.OzoneAclException;
@@ -34,21 +37,17 @@ import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
import org.apache.hadoop.ozone.web.request.OzoneQuota;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.test.LambdaTestUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
-
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.UUID;
+import org.apache.commons.lang3.RandomStringUtils;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
+import org.junit.AfterClass;
import static org.junit.Assert.assertTrue;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
/**
* Test for Ozone Manager ACLs.
@@ -123,7 +122,7 @@ public class TestOmAcls {
createVolumeArgs.setAdminName(adminUser);
createVolumeArgs.setQuota(new OzoneQuota(i, OzoneQuota.Units.GB));
logCapturer.clearOutput();
- LambdaTestUtils.intercept(IOException.class, "Volume creation failed",
+ OzoneTestUtils.expectOmException(ResultCodes.INTERNAL_ERROR,
() -> storageHandler.createVolume(createVolumeArgs));
assertTrue(logCapturer.getOutput().contains("doesn't have CREATE " +
"permission to access volume"));
@@ -132,7 +131,7 @@ public class TestOmAcls {
bucketArgs.setAddAcls(new LinkedList<>());
bucketArgs.setRemoveAcls(new LinkedList<>());
bucketArgs.setStorageType(StorageType.DISK);
- LambdaTestUtils.intercept(IOException.class, "Bucket creation failed",
+ OzoneTestUtils.expectOmException(ResultCodes.INTERNAL_ERROR,
() -> storageHandler.createBucket(bucketArgs));
assertTrue(logCapturer.getOutput().contains("doesn't have CREATE " +
"permission to access bucket"));
@@ -156,7 +155,7 @@ public class TestOmAcls {
// write a key without specifying size at all
String keyName = "testKey";
KeyArgs keyArgs = new KeyArgs(keyName, bucketArgs);
- LambdaTestUtils.intercept(IOException.class, "Create key failed",
+ OzoneTestUtils.expectOmException(ResultCodes.INTERNAL_ERROR,
() -> storageHandler.newKeyWriter(keyArgs));
assertTrue(logCapturer.getOutput().contains("doesn't have READ permission" +
" to access key"));
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
index eb4421a..5eefce1 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestOzoneManager.java
@@ -16,53 +16,73 @@
*/
package org.apache.hadoop.ozone.om;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.net.InetSocketAddress;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import java.text.ParseException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
-import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.hdds.HddsConfigKeys;
+import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.StorageType;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.scm.server.SCMStorageConfig;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.server.datanode.ObjectStoreHandler;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.OzoneConfigKeys;
-import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.OzoneConsts;
-import org.apache.hadoop.ozone.common.BlockGroup;
+import org.apache.hadoop.ozone.OzoneTestUtils;
import org.apache.hadoop.ozone.client.rest.OzoneException;
+import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
-import org.apache.hadoop.ozone.protocol.proto
- .OzoneManagerProtocolProtos.ServicePort;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
import org.apache.hadoop.ozone.util.OzoneVersionInfo;
import org.apache.hadoop.ozone.web.handlers.BucketArgs;
import org.apache.hadoop.ozone.web.handlers.KeyArgs;
+import org.apache.hadoop.ozone.web.handlers.ListArgs;
import org.apache.hadoop.ozone.web.handlers.UserArgs;
import org.apache.hadoop.ozone.web.handlers.VolumeArgs;
import org.apache.hadoop.ozone.web.interfaces.StorageHandler;
-import org.apache.hadoop.ozone.OzoneAcl;
import org.apache.hadoop.ozone.web.request.OzoneQuota;
import org.apache.hadoop.ozone.web.response.BucketInfo;
import org.apache.hadoop.ozone.web.response.KeyInfo;
-import org.apache.hadoop.ozone.web.response.VolumeInfo;
-import org.apache.hadoop.ozone.web.utils.OzoneUtils;
-import org.apache.hadoop.hdds.scm.ScmConfigKeys;
-import org.apache.hadoop.hdds.scm.ScmInfo;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.hadoop.ozone.protocol.proto
- .OzoneManagerProtocolProtos.Status;
-import org.apache.hadoop.ozone.web.handlers.ListArgs;
import org.apache.hadoop.ozone.web.response.ListBuckets;
import org.apache.hadoop.ozone.web.response.ListKeys;
import org.apache.hadoop.ozone.web.response.ListVolumes;
+import org.apache.hadoop.ozone.web.response.VolumeInfo;
+import org.apache.hadoop.ozone.web.utils.OzoneUtils;
+import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
import org.apache.hadoop.utils.db.Table;
import org.apache.hadoop.utils.db.Table.KeyValue;
import org.apache.hadoop.utils.db.TableIterator;
+
+import org.apache.commons.lang3.RandomStringUtils;
+import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
+import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
import org.apache.ratis.util.LifeCycle;
import org.junit.After;
import org.junit.Assert;
@@ -70,31 +90,8 @@ import org.junit.Before;
import org.junit.Ignore;
import org.junit.Rule;
import org.junit.Test;
-import org.junit.rules.ExpectedException;
import org.junit.rules.Timeout;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import java.net.InetSocketAddress;
-import java.text.ParseException;
-import java.util.LinkedList;
-import java.util.Random;
-import java.util.Set;
-import java.util.List;
-import java.util.UUID;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_ENABLED;
-import static org.apache.hadoop.ozone.OzoneConfigKeys
- .OZONE_OPEN_KEY_EXPIRE_THRESHOLD_SECONDS;
-import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
-import static org.apache.hadoop.hdds.scm.ScmConfigKeys
- .OZONE_SCM_CLIENT_ADDRESS_KEY;
-
/**
* Test Ozone Manager operation in distributed handler scenario.
*/
@@ -109,9 +106,6 @@ public class TestOzoneManager {
private String omId;
@Rule
- public ExpectedException exception = ExpectedException.none();
-
- @Rule
public Timeout timeout = new Timeout(60000);
/**
@@ -291,9 +285,8 @@ public class TestOzoneManager {
volumeArgs = new VolumeArgs(volumeName1, userArgs);
storageHandler.getVolumeInfo(volumeArgs);
Assert.fail("Volume is not deleted");
- } catch (IOException ex) {
- Assert.assertEquals("Info Volume failed, error:VOLUME_NOT_FOUND",
- ex.getMessage());
+ } catch (OMException ex) {
+ Assert.assertEquals(ResultCodes.VOLUME_NOT_FOUND, ex.getResult());
}
//delete the _AA volume, too
storageHandler.deleteVolume(new VolumeArgs(volumeName2, userArgs));
@@ -340,9 +333,8 @@ public class TestOzoneManager {
storageHandler.deleteVolume(createVolumeArgs);
Assert.fail("Expecting deletion should fail "
+ "because volume is not empty");
- } catch (IOException ex) {
- Assert.assertEquals(ex.getMessage(),
- "Delete Volume failed, error:VOLUME_NOT_EMPTY");
+ } catch (OMException ex) {
+ Assert.assertEquals(ResultCodes.VOLUME_NOT_EMPTY, ex.getResult());
}
retVolumeInfo = storageHandler.getVolumeInfo(getVolumeArgs);
Assert.assertTrue(retVolumeInfo.getVolumeName().equals(volumeName));
@@ -426,7 +418,7 @@ public class TestOzoneManager {
}
@Test
- public void testDeleteBucket() throws IOException, OzoneException {
+ public void testDeleteBucket() throws Exception {
String userName = "user" + RandomStringUtils.randomNumeric(5);
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
@@ -443,13 +435,13 @@ public class TestOzoneManager {
Assert.assertTrue(bucketInfo.getVolumeName().equals(volumeName));
Assert.assertTrue(bucketInfo.getBucketName().equals(bucketName));
storageHandler.deleteBucket(bucketArgs);
- exception.expect(IOException.class);
- exception.expectMessage("Info Bucket failed, error: BUCKET_NOT_FOUND");
- storageHandler.getBucketInfo(getBucketArgs);
+
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND,
+ () -> storageHandler.getBucketInfo(getBucketArgs));
}
@Test
- public void testDeleteNonExistingBucket() throws IOException, OzoneException {
+ public void testDeleteNonExistingBucket() throws Exception {
String userName = "user" + RandomStringUtils.randomNumeric(5);
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
@@ -467,14 +459,13 @@ public class TestOzoneManager {
Assert.assertTrue(bucketInfo.getBucketName().equals(bucketName));
BucketArgs newBucketArgs = new BucketArgs(
volumeName, bucketName + "_invalid", userArgs);
- exception.expect(IOException.class);
- exception.expectMessage("Delete Bucket failed, error:BUCKET_NOT_FOUND");
- storageHandler.deleteBucket(newBucketArgs);
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND,
+ () -> storageHandler.deleteBucket(newBucketArgs));
}
@Test
- public void testDeleteNonEmptyBucket() throws IOException, OzoneException {
+ public void testDeleteNonEmptyBucket() throws Exception {
String userName = "user" + RandomStringUtils.randomNumeric(5);
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
@@ -497,9 +488,9 @@ public class TestOzoneManager {
try (OutputStream stream = storageHandler.newKeyWriter(keyArgs)) {
stream.write(dataString.getBytes());
}
- exception.expect(IOException.class);
- exception.expectMessage("Delete Bucket failed, error:BUCKET_NOT_EMPTY");
- storageHandler.deleteBucket(bucketArgs);
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_EMPTY,
+ () -> storageHandler.deleteBucket(bucketArgs));
+
}
/**
@@ -597,7 +588,7 @@ public class TestOzoneManager {
* @throws OzoneException
*/
@Test
- public void testGetNonExistKey() throws IOException, OzoneException {
+ public void testGetNonExistKey() throws Exception {
String userName = "user" + RandomStringUtils.randomNumeric(5);
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
@@ -618,9 +609,8 @@ public class TestOzoneManager {
KeyArgs keyArgs = new KeyArgs(volumeName, bucketName, keyName, userArgs);
// try to get the key, should fail as it hasn't been created
- exception.expect(IOException.class);
- exception.expectMessage("KEY_NOT_FOUND");
- storageHandler.newKeyReader(keyArgs);
+ OzoneTestUtils.expectOmException(KEY_NOT_FOUND,
+ () -> storageHandler.newKeyReader(keyArgs));
Assert.assertEquals(1 + numKeyLookupFails,
omMetrics.getNumKeyLookupFails());
}
@@ -632,7 +622,7 @@ public class TestOzoneManager {
* @throws OzoneException
*/
@Test
- public void testDeleteKey() throws IOException, OzoneException {
+ public void testDeleteKey() throws Exception {
String userName = "user" + RandomStringUtils.randomNumeric(5);
String adminName = "admin" + RandomStringUtils.randomNumeric(5);
String volumeName = "volume" + RandomStringUtils.randomNumeric(5);
@@ -670,12 +660,8 @@ public class TestOzoneManager {
}
// Delete the key again to test deleting non-existing key.
- try {
- storageHandler.deleteKey(keyArgs);
- Assert.fail("Expected exception not thrown.");
- } catch (IOException ioe) {
- Assert.assertTrue(ioe.getMessage().contains("KEY_NOT_FOUND"));
- }
+ OzoneTestUtils.expectOmException(KEY_NOT_FOUND,
+ () -> storageHandler.deleteKey(keyArgs));
Assert.assertEquals(1 + numKeyDeleteFails,
omMetrics.getNumKeyDeletesFails());
}
@@ -697,7 +683,7 @@ public class TestOzoneManager {
long numKeyRenameFails = omMetrics.getNumKeyRenameFails();
int testRenameFails = 0;
int testRenames = 0;
- IOException ioe = null;
+ OMException omException = null;
VolumeArgs createVolumeArgs = new VolumeArgs(volumeName, userArgs);
createVolumeArgs.setUserName(userName);
@@ -715,11 +701,11 @@ public class TestOzoneManager {
try {
testRenames++;
storageHandler.renameKey(keyArgs, toKeyName);
- } catch (IOException e) {
+ } catch (OMException e) {
testRenameFails++;
- ioe = e;
+ omException = e;
}
- Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error"));
+ Assert.assertEquals(KEY_NOT_FOUND, omException.getResult());
// Write the contents of the key to be renamed
String dataString = RandomStringUtils.randomAscii(100);
@@ -739,10 +725,10 @@ public class TestOzoneManager {
// Try to get the key, should fail as it has been renamed
try {
storageHandler.newKeyReader(keyArgs);
- } catch (IOException e) {
- ioe = e;
+ } catch (OMException e) {
+ omException = e;
}
- Assert.assertTrue(ioe.getMessage().contains("KEY_NOT_FOUND"));
+ Assert.assertEquals(KEY_NOT_FOUND, omException.getResult());
// Verify the contents of the renamed key
keyArgs = new KeyArgs(toKeyName, bucketArgs);
@@ -760,22 +746,22 @@ public class TestOzoneManager {
stream.close();
testRenames++;
storageHandler.renameKey(keyArgs, toKeyName);
- } catch (IOException e) {
+ } catch (OMException e) {
testRenameFails++;
- ioe = e;
+ omException = e;
}
- Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error"));
+ Assert.assertEquals(ResultCodes.KEY_RENAME_ERROR, omException);
// Rename to empty string should fail
toKeyName = "";
try {
testRenames++;
storageHandler.renameKey(keyArgs, toKeyName);
- } catch (IOException e) {
+ } catch (OMException e) {
testRenameFails++;
- ioe = e;
+ omException = e;
}
- Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error"));
+ Assert.assertEquals(ResultCodes.KEY_RENAME_ERROR, omException);
// Rename from empty string should fail
keyArgs = new KeyArgs("", bucketArgs);
@@ -783,11 +769,11 @@ public class TestOzoneManager {
try {
testRenames++;
storageHandler.renameKey(keyArgs, toKeyName);
- } catch (IOException e) {
+ } catch (OMException e) {
testRenameFails++;
- ioe = e;
+ omException = e;
}
- Assert.assertTrue(ioe.getMessage().contains("Rename key failed, error"));
+ Assert.assertEquals(ResultCodes.KEY_RENAME_ERROR, omException);
Assert.assertEquals(numKeyRenames + testRenames,
omMetrics.getNumKeyRenames());
@@ -900,10 +886,8 @@ public class TestOzoneManager {
listBucketArgs = new ListArgs(invalidVolArgs, null, 100, null);
storageHandler.listBuckets(listBucketArgs);
Assert.fail("Expecting an error when the given volume name is invalid.");
- } catch (Exception e) {
- Assert.assertTrue(e instanceof IOException);
- Assert.assertTrue(e.getMessage()
- .contains(Status.VOLUME_NOT_FOUND.name()));
+ } catch (OMException e) {
+ Assert.assertEquals(VOLUME_NOT_FOUND, e.getResult());
}
}
@@ -913,7 +897,7 @@ public class TestOzoneManager {
* @throws OzoneException
*/
@Test
- public void testListKeys() throws IOException, OzoneException {
+ public void testListKeys() throws Exception {
ListKeys result = null;
ListArgs listKeyArgs = null;
@@ -1002,17 +986,12 @@ public class TestOzoneManager {
OzoneConsts.MAX_LISTKEYS_SIZE), e);
}
- // Provide an invalid bucket name.
- bucketArgs = new BucketArgs("invalid_bucket", createVolumeArgs);
- try {
- listKeyArgs = new ListArgs(bucketArgs, null, numKeys, null);
- storageHandler.listKeys(listKeyArgs);
- Assert.fail(
- "Expecting an error when the given bucket name is invalid.");
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- Status.BUCKET_NOT_FOUND.name(), e);
- }
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND, () -> {
+ // Provide an invalid bucket name.
+ BucketArgs bucket = new BucketArgs("invalid_bucket", createVolumeArgs);
+ ListArgs ks = new ListArgs(bucket, null, numKeys, null);
+ storageHandler.listKeys(ks);
+ });
}
@Test
@@ -1338,17 +1317,20 @@ public class TestOzoneManager {
config.set(ScmConfigKeys.OZONE_SCM_CLIENT_ADDRESS_KEY, "127.0.0.1:0");
config.set(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY,
conf.get(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_ADDRESS_KEY));
- exception.expect(OMException.class);
- exception.expectMessage("OM not initialized.");
- OzoneManager.createOm(null, config);
- OMStorage omStore = new OMStorage(config);
- omStore.setClusterId("testClusterId");
- omStore.setScmId("testScmId");
- // writes the version file properties
- omStore.initialize();
- exception.expect(OMException.class);
- exception.expectMessage("SCM version info mismatch.");
- OzoneManager.createOm(null, conf);
+
+ OzoneTestUtils.expectOmException(ResultCodes.OM_NOT_INITIALIZED, () -> {
+ OzoneManager.createOm(null, config);
+ });
+
+ OzoneTestUtils
+ .expectOmException(ResultCodes.SCM_VERSION_MISMATCH_ERROR, () -> {
+ OMStorage omStore = new OMStorage(config);
+ omStore.setClusterId("testClusterId");
+ omStore.setScmId("testScmId");
+ // writes the version file properties
+ omStore.initialize();
+ OzoneManager.createOm(null, conf);
+ });
}
@Test
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
index 4bf7d52..796a7a9 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/ozShell/TestOzoneShell.java
@@ -56,6 +56,8 @@ import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.client.rest.RestClient;
import org.apache.hadoop.ozone.client.rpc.RpcClient;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort;
import org.apache.hadoop.ozone.web.ozShell.Shell;
@@ -72,6 +74,10 @@ import org.apache.commons.lang3.RandomStringUtils;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_REPLICATION;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.S3_BUCKET_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
@@ -125,8 +131,7 @@ public class TestOzoneShell {
@Parameterized.Parameters
public static Collection<Object[]> clientProtocol() {
Object[][] params = new Object[][] {
- {RpcClient.class},
- {RestClient.class}};
+ {RpcClient.class}};
return Arrays.asList(params);
}
@@ -297,9 +302,8 @@ public class TestOzoneShell {
try {
client.getVolumeDetails(volumeName);
fail("Get volume call should have thrown.");
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- "Info Volume failed, error:VOLUME_NOT_FOUND", e);
+ } catch (OMException e) {
+ Assert.assertEquals(VOLUME_NOT_FOUND, e.getResult());
}
@@ -324,9 +328,8 @@ public class TestOzoneShell {
try {
client.getVolumeDetails(volumeName);
fail("Get volume call should have thrown.");
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- "Info Volume failed, error:VOLUME_NOT_FOUND", e);
+ } catch (OMException e) {
+ Assert.assertEquals(VOLUME_NOT_FOUND, e.getResult());
}
}
@@ -368,7 +371,7 @@ public class TestOzoneShell {
// get info for non-exist volume
args = new String[] {"volume", "info", url + "/invalid-volume"};
- executeWithError(shell, args, "VOLUME_NOT_FOUND");
+ executeWithError(shell, args, VOLUME_NOT_FOUND);
}
@Test
@@ -434,16 +437,34 @@ public class TestOzoneShell {
// test error conditions
args = new String[] {"volume", "update", url + "/invalid-volume",
"--user", newUser};
- executeWithError(shell, args, "Info Volume failed, error:VOLUME_NOT_FOUND");
+ executeWithError(shell, args, ResultCodes.VOLUME_NOT_FOUND);
err.reset();
args = new String[] {"volume", "update", url + "/invalid-volume",
"--quota", "500MB"};
- executeWithError(shell, args, "Info Volume failed, error:VOLUME_NOT_FOUND");
+ executeWithError(shell, args, ResultCodes.VOLUME_NOT_FOUND);
+ }
+
+ /**
+ * Execute command, assert exception message and returns true if error
+ * was thrown.
+ */
+ private void executeWithError(Shell ozoneShell, String[] args,
+ OMException.ResultCodes code) {
+
+ try {
+ execute(ozoneShell, args);
+ fail("Exception is expected from command execution " + Arrays
+ .asList(args));
+ } catch (Exception ex) {
+ Assert.assertEquals(OMException.class, ex.getCause().getClass());
+ Assert.assertEquals(code, ((OMException) ex.getCause()).getResult());
+ }
+
}
/**
- * Execute command, assert exeception message and returns true if error
+ * Execute command, assert exception message and returns true if error
* was thrown.
*/
private void executeWithError(Shell ozoneShell, String[] args,
@@ -654,7 +675,7 @@ public class TestOzoneShell {
// test create a bucket in a non-exist volume
args = new String[] {"bucket", "create",
url + "/invalid-volume/" + bucketName};
- executeWithError(shell, args, "Info Volume failed, error:VOLUME_NOT_FOUND");
+ executeWithError(shell, args, VOLUME_NOT_FOUND);
// test createBucket with invalid bucket name
args = new String[] {"bucket", "create",
@@ -680,22 +701,20 @@ public class TestOzoneShell {
try {
vol.getBucket(bucketName);
fail("Get bucket should have thrown.");
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- "Info Bucket failed, error: BUCKET_NOT_FOUND", e);
+ } catch (OMException e) {
+ Assert.assertEquals(BUCKET_NOT_FOUND, e.getResult());
}
// test delete bucket in a non-exist volume
args = new String[] {"bucket", "delete",
url + "/invalid-volume" + "/" + bucketName};
- executeWithError(shell, args, "Info Volume failed, error:VOLUME_NOT_FOUND");
+ executeWithError(shell, args, VOLUME_NOT_FOUND);
err.reset();
// test delete non-exist bucket
args = new String[] {"bucket", "delete",
url + "/" + vol.getName() + "/invalid-bucket"};
- executeWithError(shell, args,
- "Delete Bucket failed, error:BUCKET_NOT_FOUND");
+ executeWithError(shell, args, BUCKET_NOT_FOUND);
}
@Test
@@ -724,7 +743,7 @@ public class TestOzoneShell {
args = new String[] {"bucket", "info",
url + "/" + vol.getName() + "/invalid-bucket" + bucketName};
executeWithError(shell, args,
- "Info Bucket failed, error: BUCKET_NOT_FOUND");
+ ResultCodes.BUCKET_NOT_FOUND);
}
@Test
@@ -768,8 +787,7 @@ public class TestOzoneShell {
args = new String[] {"bucket", "update",
url + "/" + vol.getName() + "/invalid-bucket", "--addAcl",
"user:frodo:rw"};
- executeWithError(shell, args,
- "Info Bucket failed, error: BUCKET_NOT_FOUND");
+ executeWithError(shell, args, BUCKET_NOT_FOUND);
}
@Test
@@ -885,8 +903,7 @@ public class TestOzoneShell {
args = new String[] {"key", "put",
url + "/" + volumeName + "/invalid-bucket/" + keyName,
createTmpFile()};
- executeWithError(shell, args,
- "Info Bucket failed, error: BUCKET_NOT_FOUND");
+ executeWithError(shell, args, BUCKET_NOT_FOUND);
}
@Test
@@ -955,14 +972,13 @@ public class TestOzoneShell {
// test delete key in a non-exist bucket
args = new String[] {"key", "delete",
url + "/" + volumeName + "/invalid-bucket/" + keyName};
- executeWithError(shell, args,
- "Info Bucket failed, error: BUCKET_NOT_FOUND");
+ executeWithError(shell, args, BUCKET_NOT_FOUND);
err.reset();
// test delete a non-exist key in bucket
args = new String[] {"key", "delete",
url + "/" + volumeName + "/" + bucketName + "/invalid-key"};
- executeWithError(shell, args, "Delete key failed, error:KEY_NOT_FOUND");
+ executeWithError(shell, args, KEY_NOT_FOUND);
}
@Test
@@ -1025,7 +1041,7 @@ public class TestOzoneShell {
// verify the response output
// get the non-exist key info should be failed
- executeWithError(shell, args, "Lookup key failed, error:KEY_NOT_FOUND");
+ executeWithError(shell, args, KEY_NOT_FOUND);
}
@Test
@@ -1052,7 +1068,7 @@ public class TestOzoneShell {
output.contains(OzoneConsts.OZONE_TIME_ZONE));
args = new String[] {"key", "info",
url + "/" + volumeName + "/" + bucketName + "/" + keyNameOnly};
- executeWithError(shell, args, "Lookup key failed, error:KEY_NOT_FOUND");
+ executeWithError(shell, args, KEY_NOT_FOUND);
out.reset();
err.reset();
}
@@ -1185,7 +1201,7 @@ public class TestOzoneShell {
//Trying to get map for an unknown bucket
args = new String[] {setOmAddress, "bucket", "path",
"unknownbucket"};
- executeWithError(shell, args, "S3_BUCKET_NOT_FOUND");
+ executeWithError(shell, args, S3_BUCKET_NOT_FOUND);
// No bucket name
args = new String[] {setOmAddress, "bucket", "path"};
@@ -1193,7 +1209,7 @@ public class TestOzoneShell {
// Invalid bucket name
args = new String[] {setOmAddress, "bucket", "path", "/asd/multipleslash"};
- executeWithError(shell, args, "S3_BUCKET_NOT_FOUND");
+ executeWithError(shell, args, S3_BUCKET_NOT_FOUND);
}
@Test
@@ -1371,13 +1387,13 @@ public class TestOzoneShell {
.orElseThrow(IllegalStateException::new);
}
- private static void assertKeyNotExists(OzoneBucket bucket, String keyName) {
+ private static void assertKeyNotExists(OzoneBucket bucket, String keyName)
+ throws IOException {
try {
bucket.getKey(keyName);
fail(String.format("Key %s should not exist, but it does", keyName));
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- "Lookup key failed, error:KEY_NOT_FOUND", e);
+ } catch (OMException e) {
+ Assert.assertEquals(KEY_NOT_FOUND, e.getResult());
}
}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
index 1882584..c4f10f7 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestKeys.java
@@ -50,13 +50,13 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.om.OzoneManager;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .Status;
+
import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.test.GenericTestUtils;
@@ -279,6 +279,7 @@ public class TestKeys {
getMultiPartKey(delimiter)));
}
+ @SuppressWarnings("emptyblock")
static void runTestPutKey(PutHelper helper) throws Exception {
final ClientProtocol helperClient = helper.client;
helper.putKey();
@@ -301,29 +302,20 @@ public class TestKeys {
Assert.assertEquals(2, keyList.size());
// test new put key with invalid volume/bucket name
- try {
- ozoneOutputStream = helperClient
+ OzoneTestUtils.expectOmException(ResultCodes.VOLUME_NOT_FOUND, () -> {
+
+ try (OzoneOutputStream oos = helperClient
.createKey("invalid-volume", helper.getBucket().getName(), newkeyName,
- 0, replicationType, replicationFactor, new HashMap<>());
- ozoneOutputStream.close();
- fail("Put key should have thrown"
- + " when using invalid volume name.");
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- Status.VOLUME_NOT_FOUND.toString(), e);
- }
+ 0, replicationType, replicationFactor, new HashMap<>())) {
+ }
+ });
- try {
- ozoneOutputStream = helperClient
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND, () -> {
+ try (OzoneOutputStream oos = helperClient
.createKey(helper.getVol().getName(), "invalid-bucket", newkeyName, 0,
- replicationType, replicationFactor, new HashMap<>());
- ozoneOutputStream.close();
- fail("Put key should have thrown "
- + "when using invalid bucket name.");
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- Status.BUCKET_NOT_FOUND.toString(), e);
- }
+ replicationType, replicationFactor, new HashMap<>())) {
+ }
+ });
}
private static void restartDatanode(MiniOzoneCluster cluster, int datanodeIdx)
@@ -428,23 +420,13 @@ public class TestKeys {
originalHash, downloadedHash2);
// test new get key with invalid volume/bucket name
- try {
- helperClient.getKey(
- "invalid-volume", helper.getBucket().getName(), keyName);
- fail("Get key should have thrown " + "when using invalid volume name.");
- } catch (IOException e) {
- GenericTestUtils
- .assertExceptionContains(Status.KEY_NOT_FOUND.toString(), e);
- }
+ OzoneTestUtils.expectOmException(ResultCodes.KEY_NOT_FOUND,
+ () -> helperClient.getKey(
+ "invalid-volume", helper.getBucket().getName(), keyName));
- try {
- helperClient.getKey(
- helper.getVol().getName(), "invalid-bucket", keyName);
- fail("Get key should have thrown " + "when using invalid bucket name.");
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- Status.KEY_NOT_FOUND.toString(), e);
- }
+ OzoneTestUtils.expectOmException(ResultCodes.KEY_NOT_FOUND,
+ () -> helperClient.getKey(
+ helper.getVol().getName(), "invalid-bucket", keyName));
}
}
@@ -462,13 +444,9 @@ public class TestKeys {
assertNotNull(helper.getFile());
helper.getBucket().deleteKey(keyName);
- try {
+ OzoneTestUtils.expectOmException(ResultCodes.KEY_NOT_FOUND, () -> {
helper.getBucket().getKey(keyName);
- fail("Get Key on a deleted key should have thrown");
- } catch (IOException ex) {
- GenericTestUtils.assertExceptionContains(
- Status.KEY_NOT_FOUND.toString(), ex);
- }
+ });
}
@Test
@@ -546,23 +524,15 @@ public class TestKeys {
keyList2.size() == 1 && keyList2.get(0).getName().equals("list-key2"));
// test new list keys with invalid volume/bucket name
- try {
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND, () -> {
helperClient.listKeys("invalid-volume", helper.getBucket().getName(),
null, null, 100);
- fail("List keys should have thrown when using invalid volume name.");
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- Status.BUCKET_NOT_FOUND.toString(), e);
- }
+ });
- try {
+ OzoneTestUtils.expectOmException(ResultCodes.BUCKET_NOT_FOUND, () -> {
helperClient.listKeys(helper.getVol().getName(), "invalid-bucket", null,
null, 100);
- fail("List keys should have thrown when using invalid bucket name.");
- } catch (IOException e) {
- GenericTestUtils.assertExceptionContains(
- Status.BUCKET_NOT_FOUND.toString(), e);
- }
+ });
}
@Test
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
index f4b013f..e24e7db 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolume.java
@@ -18,43 +18,43 @@
package org.apache.hadoop.ozone.web.client;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.RandomStringUtils;
+import java.io.File;
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
import org.apache.hadoop.hdds.client.OzoneQuota;
-import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
+import org.apache.hadoop.ozone.MiniOzoneCluster;
+import org.apache.hadoop.ozone.OzoneTestUtils;
+import org.apache.hadoop.ozone.client.OzoneVolume;
import org.apache.hadoop.ozone.client.VolumeArgs;
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
+import org.apache.hadoop.ozone.client.rest.OzoneException;
import org.apache.hadoop.ozone.client.rest.RestClient;
import org.apache.hadoop.ozone.client.rpc.RpcClient;
-import org.apache.hadoop.ozone.client.rest.OzoneException;
-import org.apache.hadoop.ozone.client.OzoneVolume;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.Time;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.RandomStringUtils;
import org.apache.log4j.Level;
import org.apache.log4j.Logger;
import org.junit.AfterClass;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
import org.junit.Before;
import org.junit.BeforeClass;
-import org.junit.Assert;
-import org.junit.Test;
import org.junit.Ignore;
+import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
-import java.io.File;
-import java.io.IOException;
-import java.text.ParseException;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.List;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
/**
* Test Ozone Volumes Lifecycle.
*/
@@ -67,8 +67,7 @@ public class TestVolume {
@Parameterized.Parameters
public static Collection<Object[]> clientProtocol() {
Object[][] params = new Object[][] {
- {RpcClient.class},
- {RestClient.class}};
+ {RpcClient.class}};
return Arrays.asList(params);
}
@@ -160,20 +159,16 @@ public class TestVolume {
}
@Test
- public void testCreateDuplicateVolume() throws OzoneException, IOException {
+ public void testCreateDuplicateVolume() throws Exception {
runTestCreateDuplicateVolume(client);
}
static void runTestCreateDuplicateVolume(ClientProtocol clientProtocol)
- throws OzoneException, IOException {
- try {
- clientProtocol.createVolume("testvol");
- clientProtocol.createVolume("testvol");
- assertFalse(true);
- } catch (IOException ioe) {
- Assert.assertTrue(ioe.getMessage()
- .contains("Volume creation failed, error:VOLUME_ALREADY_EXISTS"));
- }
+ throws Exception {
+
+ clientProtocol.createVolume("testvol");
+ OzoneTestUtils.expectOmException(ResultCodes.VOLUME_ALREADY_EXISTS,
+ () -> clientProtocol.createVolume("testvol"));
}
@Test
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
index 186eed9..27074a3 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/web/client/TestVolumeRatis.java
@@ -107,7 +107,7 @@ public class TestVolumeRatis {
}
@Test
- public void testCreateDuplicateVolume() throws OzoneException, IOException {
+ public void testCreateDuplicateVolume() throws Exception {
TestVolume.runTestCreateDuplicateVolume(client);
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
index c108329..c936354 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/BucketManagerImpl.java
@@ -91,13 +91,13 @@ public class BucketManagerImpl implements BucketManager {
if (metadataManager.getVolumeTable().get(volumeKey) == null) {
LOG.debug("volume: {} not found ", volumeName);
throw new OMException("Volume doesn't exist",
- OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ OMException.ResultCodes.VOLUME_NOT_FOUND);
}
//Check if bucket already exists
if (metadataManager.getBucketTable().get(bucketKey) != null) {
LOG.debug("bucket: {} already exists ", bucketName);
throw new OMException("Bucket already exist",
- OMException.ResultCodes.FAILED_BUCKET_ALREADY_EXISTS);
+ OMException.ResultCodes.BUCKET_ALREADY_EXISTS);
}
OmBucketInfo omBucketInfo = OmBucketInfo.newBuilder()
@@ -144,7 +144,7 @@ public class BucketManagerImpl implements BucketManager {
LOG.debug("bucket: {} not found in volume: {}.", bucketName,
volumeName);
throw new OMException("Bucket not found",
- OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
+ OMException.ResultCodes.BUCKET_NOT_FOUND);
}
return value;
} catch (IOException | DBException ex) {
@@ -178,7 +178,7 @@ public class BucketManagerImpl implements BucketManager {
if (oldBucketInfo == null) {
LOG.debug("bucket: {} not found ", bucketName);
throw new OMException("Bucket doesn't exist",
- OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
+ OMException.ResultCodes.BUCKET_NOT_FOUND);
}
OmBucketInfo.Builder bucketInfoBuilder = OmBucketInfo.newBuilder();
bucketInfoBuilder.setVolumeName(oldBucketInfo.getVolumeName())
@@ -270,13 +270,13 @@ public class BucketManagerImpl implements BucketManager {
if (metadataManager.getBucketTable().get(bucketKey) == null) {
LOG.debug("bucket: {} not found ", bucketName);
throw new OMException("Bucket doesn't exist",
- OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
+ OMException.ResultCodes.BUCKET_NOT_FOUND);
}
//Check if bucket is empty
if (!metadataManager.isBucketEmpty(volumeName, bucketName)) {
LOG.debug("bucket: {} is not empty ", bucketName);
throw new OMException("Bucket is not empty",
- OMException.ResultCodes.FAILED_BUCKET_NOT_EMPTY);
+ OMException.ResultCodes.BUCKET_NOT_EMPTY);
}
metadataManager.getBucketTable().delete(bucketKey);
} catch (IOException ex) {
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index 3fa46f9..c9ac3f0 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -161,13 +161,13 @@ public class KeyManagerImpl implements KeyManager {
if (metadataManager.getVolumeTable().get(volumeKey) == null) {
LOG.error("volume not found: {}", volumeName);
throw new OMException("Volume not found",
- OMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ OMException.ResultCodes.VOLUME_NOT_FOUND);
}
//Check if bucket already exists
if (metadataManager.getBucketTable().get(bucketKey) == null) {
LOG.error("bucket not found: {}/{} ", volumeName, bucketName);
throw new OMException("Bucket not found",
- OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
+ OMException.ResultCodes.BUCKET_NOT_FOUND);
}
}
@@ -185,7 +185,7 @@ public class KeyManagerImpl implements KeyManager {
if (metadataManager.getBucketTable().get(bucketKey) == null) {
LOG.error("bucket not found: {}/{} ", volumeName, bucketName);
throw new OMException("Bucket not found",
- OMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
+ ResultCodes.BUCKET_NOT_FOUND);
}
}
@@ -205,7 +205,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.error("Allocate block for a key not in open status in meta store" +
" /{}/{}/{} with ID {}", volumeName, bucketName, keyName, clientID);
throw new OMException("Open Key not found",
- OMException.ResultCodes.FAILED_KEY_NOT_FOUND);
+ OMException.ResultCodes.KEY_NOT_FOUND);
}
AllocatedBlock allocatedBlock;
@@ -288,7 +288,8 @@ public class KeyManagerImpl implements KeyManager {
multipartKey);
if (partKeyInfo == null) {
throw new OMException("No such Multipart upload is with specified " +
- "uploadId " + uploadID, ResultCodes.NO_SUCH_MULTIPART_UPLOAD);
+ "uploadId " + uploadID,
+ ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
} else {
factor = partKeyInfo.getFactor();
type = partKeyInfo.getType();
@@ -386,7 +387,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.warn("Cannot allocate key. The generated open key id is already" +
"used for the same key which is currently being written.");
throw new OMException("Cannot allocate key. Not able to get a valid" +
- "open key id.", OMException.ResultCodes.FAILED_KEY_ALLOCATION);
+ "open key id.", ResultCodes.KEY_ALLOCATION_ERROR);
}
metadataManager.getOpenKeyTable().put(openKey, keyInfo);
LOG.debug("Key {} allocated in volume {} bucket {}",
@@ -398,7 +399,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.error("Key open failed for volume:{} bucket:{} key:{}",
volumeName, bucketName, keyName, ex);
throw new OMException(ex.getMessage(),
- OMException.ResultCodes.FAILED_KEY_ALLOCATION);
+ ResultCodes.KEY_ALLOCATION_ERROR);
} finally {
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
}
@@ -447,7 +448,7 @@ public class KeyManagerImpl implements KeyManager {
OmKeyInfo keyInfo = metadataManager.getOpenKeyTable().get(openKey);
if (keyInfo == null) {
throw new OMException("Commit a key without corresponding entry " +
- objectKey, ResultCodes.FAILED_KEY_NOT_FOUND);
+ objectKey, ResultCodes.KEY_NOT_FOUND);
}
keyInfo.setDataSize(args.getDataSize());
keyInfo.setModificationTime(Time.now());
@@ -468,7 +469,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.error("Key commit failed for volume:{} bucket:{} key:{}",
volumeName, bucketName, keyName, ex);
throw new OMException(ex.getMessage(),
- OMException.ResultCodes.FAILED_KEY_ALLOCATION);
+ ResultCodes.KEY_ALLOCATION_ERROR);
} finally {
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
}
@@ -489,7 +490,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.debug("volume:{} bucket:{} Key:{} not found",
volumeName, bucketName, keyName);
throw new OMException("Key not found",
- OMException.ResultCodes.FAILED_KEY_NOT_FOUND);
+ OMException.ResultCodes.KEY_NOT_FOUND);
}
if (grpcBlockTokenEnabled) {
String remoteUser = getRemoteUser().getShortUserName();
@@ -507,7 +508,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.debug("Get key failed for volume:{} bucket:{} key:{}",
volumeName, bucketName, keyName, ex);
throw new OMException(ex.getMessage(),
- OMException.ResultCodes.FAILED_KEY_NOT_FOUND);
+ OMException.ResultCodes.KEY_NOT_FOUND);
} finally {
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
}
@@ -524,7 +525,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}",
volumeName, bucketName, fromKeyName, toKeyName);
throw new OMException("Key name is empty",
- ResultCodes.FAILED_INVALID_KEY_NAME);
+ ResultCodes.INVALID_KEY_NAME);
}
metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
@@ -540,7 +541,7 @@ public class KeyManagerImpl implements KeyManager {
+ "Key: {} not found.", volumeName, bucketName, fromKeyName,
toKeyName, fromKeyName);
throw new OMException("Key not found",
- OMException.ResultCodes.FAILED_KEY_NOT_FOUND);
+ OMException.ResultCodes.KEY_NOT_FOUND);
}
// A rename is a no-op if the target and source name is same.
@@ -559,7 +560,7 @@ public class KeyManagerImpl implements KeyManager {
+ "Key: {} already exists.", volumeName, bucketName,
fromKeyName, toKeyName, toKeyName);
throw new OMException("Key not found",
- OMException.ResultCodes.FAILED_KEY_ALREADY_EXISTS);
+ OMException.ResultCodes.KEY_ALREADY_EXISTS);
}
fromKeyValue.setKeyName(toKeyName);
@@ -572,10 +573,13 @@ public class KeyManagerImpl implements KeyManager {
store.commitBatchOperation(batch);
}
} catch (IOException ex) {
+ if (ex instanceof OMException) {
+ throw ex;
+ }
LOG.error("Rename key failed for volume:{} bucket:{} fromKey:{} toKey:{}",
volumeName, bucketName, fromKeyName, toKeyName, ex);
throw new OMException(ex.getMessage(),
- ResultCodes.FAILED_KEY_RENAME);
+ ResultCodes.KEY_RENAME_ERROR);
} finally {
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
}
@@ -594,7 +598,7 @@ public class KeyManagerImpl implements KeyManager {
OmKeyInfo keyInfo = metadataManager.getKeyTable().get(objectKey);
if (keyInfo == null) {
throw new OMException("Key not found",
- OMException.ResultCodes.FAILED_KEY_NOT_FOUND);
+ OMException.ResultCodes.KEY_NOT_FOUND);
} else {
// directly delete key with no blocks from db. This key need not be
// moved to deleted table.
@@ -613,7 +617,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.error(String.format("Delete key failed for volume:%s "
+ "bucket:%s key:%s", volumeName, bucketName, keyName), ex);
throw new OMException(ex.getMessage(), ex,
- ResultCodes.FAILED_KEY_DELETION);
+ ResultCodes.KEY_DELETION_ERROR);
} finally {
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
}
@@ -737,7 +741,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.error("Initiate Multipart upload Failed for volume:{} bucket:{} " +
"key:{}", volumeName, bucketName, keyName, ex);
throw new OMException(ex.getMessage(),
- OMException.ResultCodes.INITIATE_MULTIPART_UPLOAD_FAILED);
+ ResultCodes.INITIATE_MULTIPART_UPLOAD_ERROR);
} finally {
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
}
@@ -782,7 +786,7 @@ public class KeyManagerImpl implements KeyManager {
// Move this part to delete table.
metadataManager.getDeletedTable().put(partName, keyInfo);
throw new OMException("No such Multipart upload is with specified " +
- "uploadId " + uploadID, ResultCodes.NO_SUCH_MULTIPART_UPLOAD);
+ "uploadId " + uploadID, ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
} else {
PartKeyInfo oldPartKeyInfo =
multipartKeyInfo.getPartKeyInfo(partNumber);
@@ -822,7 +826,8 @@ public class KeyManagerImpl implements KeyManager {
LOG.error("Upload part Failed: volume:{} bucket:{} " +
"key:{} PartNumber: {}", volumeName, bucketName, keyName,
partNumber, ex);
- throw new OMException(ex.getMessage(), ResultCodes.UPLOAD_PART_FAILED);
+ throw new OMException(ex.getMessage(),
+ ResultCodes.MULTIPART_UPLOAD_PARTFILE_ERROR);
} finally {
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
}
@@ -855,7 +860,7 @@ public class KeyManagerImpl implements KeyManager {
if (multipartKeyInfo == null) {
throw new OMException("Complete Multipart Upload Failed: volume: " +
volumeName + "bucket: " + bucketName + "key: " + keyName,
- ResultCodes.NO_SUCH_MULTIPART_UPLOAD);
+ ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
}
TreeMap<Integer, PartKeyInfo> partKeyInfoMap = multipartKeyInfo
.getPartKeyInfoMap();
@@ -979,7 +984,7 @@ public class KeyManagerImpl implements KeyManager {
LOG.error("Complete Multipart Upload Failed: volume: " + volumeName +
"bucket: " + bucketName + "key: " + keyName, ex);
throw new OMException(ex.getMessage(), ResultCodes
- .COMPLETE_MULTIPART_UPLOAD_FAILED);
+ .COMPLETE_MULTIPART_UPLOAD_ERROR);
} finally {
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
}
@@ -1013,7 +1018,7 @@ public class KeyManagerImpl implements KeyManager {
"such uploadID:" + uploadID);
throw new OMException("Abort Multipart Upload Failed: volume: " +
volumeName + "bucket: " + bucketName + "key: " + keyName,
- ResultCodes.NO_SUCH_MULTIPART_UPLOAD);
+ ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
} else {
// Move all the parts to delete table
TreeMap<Integer, PartKeyInfo> partKeyInfoMap = multipartKeyInfo
@@ -1072,7 +1077,7 @@ public class KeyManagerImpl implements KeyManager {
if (multipartKeyInfo == null) {
throw new OMException("No Such Multipart upload exists for this key.",
- ResultCodes.NO_SUCH_MULTIPART_UPLOAD);
+ ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
} else {
TreeMap<Integer, PartKeyInfo> partKeyInfoMap =
multipartKeyInfo.getPartKeyInfoMap();
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
index 7a75ec8..829f35e 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java
@@ -59,8 +59,6 @@ import org.eclipse.jetty.util.StringUtil;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import javax.ws.rs.HEAD;
-
/**
* Ozone metadata manager interface.
*/
@@ -434,13 +432,13 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
List<OmBucketInfo> result = new ArrayList<>();
if (Strings.isNullOrEmpty(volumeName)) {
throw new OMException("Volume name is required.",
- ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ ResultCodes.VOLUME_NOT_FOUND);
}
String volumeNameBytes = getVolumeKey(volumeName);
if (volumeTable.get(volumeNameBytes) == null) {
throw new OMException("Volume " + volumeName + " not found.",
- ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ ResultCodes.VOLUME_NOT_FOUND);
}
String startKey;
@@ -496,18 +494,18 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
List<OmKeyInfo> result = new ArrayList<>();
if (Strings.isNullOrEmpty(volumeName)) {
throw new OMException("Volume name is required.",
- ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ ResultCodes.VOLUME_NOT_FOUND);
}
if (Strings.isNullOrEmpty(bucketName)) {
throw new OMException("Bucket name is required.",
- ResultCodes.FAILED_BUCKET_NOT_FOUND);
+ ResultCodes.BUCKET_NOT_FOUND);
}
String bucketNameBytes = getBucketKey(volumeName, bucketName);
if (getBucketTable().get(bucketNameBytes) == null) {
throw new OMException("Bucket " + bucketName + " not found.",
- ResultCodes.FAILED_BUCKET_NOT_FOUND);
+ ResultCodes.BUCKET_NOT_FOUND);
}
String seekKey;
@@ -558,7 +556,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
VolumeList volumes;
if (StringUtil.isBlank(userName)) {
throw new OMException("User name is required to list Volumes.",
- ResultCodes.FAILED_USER_NOT_FOUND);
+ ResultCodes.USER_NOT_FOUND);
}
volumes = getVolumesByUser(userName);
@@ -587,7 +585,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
// this probably means om db is corrupted or some entries are
// accidentally removed.
throw new OMException("Volume info not found for " + volumeName,
- ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ ResultCodes.VOLUME_NOT_FOUND);
}
result.add(volumeArgs);
}
@@ -609,7 +607,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager {
} catch (IOException e) {
throw new OMException("Unable to get volumes info by the given user, "
+ "metadata might be corrupted", e,
- ResultCodes.FAILED_METADATA_ERROR);
+ ResultCodes.METADATA_ERROR);
}
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java
index 43ceb80..d0d84f8 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/S3BucketManagerImpl.java
@@ -25,6 +25,8 @@ import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
import org.apache.hadoop.ozone.om.helpers.OmVolumeArgs;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_ALREADY_EXISTS;
import org.apache.logging.log4j.util.Strings;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -34,7 +36,6 @@ import java.nio.charset.StandardCharsets;
import java.util.Objects;
import static org.apache.hadoop.ozone.OzoneConsts.OM_S3_VOLUME_PREFIX;
-import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.FAILED_VOLUME_ALREADY_EXISTS;
/**
* S3 Bucket Manager, this class maintains a mapping between S3 Bucket and Ozone
@@ -168,7 +169,7 @@ public class S3BucketManagerImpl implements S3BucketManager {
volumeManager.createVolume(args);
} catch (OMException exp) {
newVolumeCreate = false;
- if (exp.getResult().compareTo(FAILED_VOLUME_ALREADY_EXISTS) == 0) {
+ if (exp.getResult().compareTo(VOLUME_ALREADY_EXISTS) == 0) {
if (LOG.isDebugEnabled()) {
LOG.debug("Volume already exists. {}", exp.getMessage());
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java
index 9bf8de1..a366ee2 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/VolumeManagerImpl.java
@@ -72,7 +72,7 @@ public class VolumeManagerImpl implements VolumeManager {
// Check the volume count
if (prevVolList.size() >= maxUserVolumeCount) {
LOG.debug("Too many volumes for user:{}", owner);
- throw new OMException(ResultCodes.FAILED_TOO_MANY_USER_VOLUMES);
+ throw new OMException(ResultCodes.USER_TOO_MANY_VOLUMES);
}
// Add the new volume to the list
@@ -93,7 +93,7 @@ public class VolumeManagerImpl implements VolumeManager {
prevVolList.addAll(volumeList.getVolumeNamesList());
} else {
LOG.debug("volume:{} not found for user:{}");
- throw new OMException(ResultCodes.FAILED_USER_NOT_FOUND);
+ throw new OMException(ResultCodes.USER_NOT_FOUND);
}
// Remove the volume from the list
@@ -125,7 +125,7 @@ public class VolumeManagerImpl implements VolumeManager {
// Check of the volume already exists
if (volumeInfo != null) {
LOG.debug("volume:{} already exists", args.getVolume());
- throw new OMException(ResultCodes.FAILED_VOLUME_ALREADY_EXISTS);
+ throw new OMException(ResultCodes.VOLUME_ALREADY_EXISTS);
}
try (BatchOperation batch = metadataManager.getStore()
@@ -173,7 +173,7 @@ public class VolumeManagerImpl implements VolumeManager {
if (volumeArgs == null) {
LOG.debug("Changing volume ownership failed for user:{} volume:{}",
owner, volume);
- throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ throw new OMException(ResultCodes.VOLUME_NOT_FOUND);
}
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
@@ -229,7 +229,7 @@ public class VolumeManagerImpl implements VolumeManager {
metadataManager.getVolumeTable().get(dbVolumeKey);
if (volumeArgs == null) {
LOG.debug("volume:{} does not exist", volume);
- throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ throw new OMException(ResultCodes.VOLUME_NOT_FOUND);
}
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
@@ -270,7 +270,7 @@ public class VolumeManagerImpl implements VolumeManager {
metadataManager.getVolumeTable().get(dbVolumeKey);
if (volumeArgs == null) {
LOG.debug("volume:{} does not exist", volume);
- throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ throw new OMException(ResultCodes.VOLUME_NOT_FOUND);
}
return volumeArgs;
@@ -309,12 +309,12 @@ public class VolumeManagerImpl implements VolumeManager {
metadataManager.getVolumeTable().get(dbVolumeKey);
if (volumeArgs == null) {
LOG.debug("volume:{} does not exist", volume);
- throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ throw new OMException(ResultCodes.VOLUME_NOT_FOUND);
}
if (!metadataManager.isVolumeEmpty(volume)) {
LOG.debug("volume:{} is not empty", volume);
- throw new OMException(ResultCodes.FAILED_VOLUME_NOT_EMPTY);
+ throw new OMException(ResultCodes.VOLUME_NOT_EMPTY);
}
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
// delete the volume from the owner list
@@ -361,7 +361,7 @@ public class VolumeManagerImpl implements VolumeManager {
metadataManager.getVolumeTable().get(dbVolumeKey);
if (volumeArgs == null) {
LOG.debug("volume:{} does not exist", volume);
- throw new OMException(ResultCodes.FAILED_VOLUME_NOT_FOUND);
+ throw new OMException(ResultCodes.VOLUME_NOT_FOUND);
}
Preconditions.checkState(volume.equals(volumeArgs.getVolume()));
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
index 5beea2e..35010ab 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/protocolPB/OzoneManagerRequestHandler.java
@@ -17,7 +17,6 @@
package org.apache.hadoop.ozone.protocolPB;
-import com.google.common.collect.Lists;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
@@ -43,138 +42,79 @@ import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
import org.apache.hadoop.ozone.om.helpers.ServiceInfo;
import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
-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
- .CheckVolumeAccessRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .CheckVolumeAccessResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .CommitKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .CommitKeyResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .CreateBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .CreateBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .CreateKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .CreateKeyResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .CreateVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .CreateVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .DeleteBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .DeleteBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .DeleteKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .DeleteKeyResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .DeleteVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .DeleteVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .InfoBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .InfoBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .InfoVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .InfoVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .KeyArgs;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .ListBucketsRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .ListBucketsResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .ListKeysRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .ListKeysResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .ListVolumeRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .ListVolumeResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .LookupKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .LookupKeyResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartCommitUploadPartRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartCommitUploadPartResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartInfoInitiateRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartInfoInitiateResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartUploadAbortRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartUploadAbortResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartUploadCompleteRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartUploadCompleteResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartUploadListPartsRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .MultipartUploadListPartsResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .OMRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .OMResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .Part;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .RenameKeyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .RenameKeyResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .S3BucketInfoRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .S3BucketInfoResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .S3CreateBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .S3CreateBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .S3DeleteBucketRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .S3DeleteBucketResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .S3ListBucketsRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .S3ListBucketsResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .ServiceListRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .ServiceListResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .SetBucketPropertyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .SetBucketPropertyResponse;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .SetVolumePropertyRequest;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
- .SetVolumePropertyResponse;
+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;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CheckVolumeAccessRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CheckVolumeAccessResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CommitKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CommitKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateBucketRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateBucketResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateVolumeRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CreateVolumeResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteBucketRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteBucketResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DeleteVolumeResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetDelegationTokenResponseProto;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetS3SecretResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoBucketRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoBucketResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoVolumeRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.InfoVolumeResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyArgs;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListBucketsRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListBucketsResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListKeysRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListKeysResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListVolumeRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ListVolumeResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LookupKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.LookupKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartCommitUploadPartRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartCommitUploadPartResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartInfoInitiateRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartInfoInitiateResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadAbortRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadAbortResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadCompleteRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadCompleteResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadListPartsRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.MultipartUploadListPartsResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.OMResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Part;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RenameKeyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RenameKeyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RenewDelegationTokenResponseProto;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3BucketInfoRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3BucketInfoResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3CreateBucketRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3CreateBucketResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3DeleteBucketRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3DeleteBucketResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3ListBucketsRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.S3ListBucketsResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServiceListResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetBucketPropertyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetBucketPropertyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetVolumePropertyRequest;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.SetVolumePropertyResponse;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
import org.apache.hadoop.ozone.security.OzoneTokenIdentifier;
import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.CancelDelegationTokenResponseProto;
import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetDelegationTokenResponseProto;
import org.apache.hadoop.security.proto.SecurityProtos.RenewDelegationTokenRequestProto;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.RenewDelegationTokenResponseProto;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.GetS3SecretResponse;
import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Status;
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.Type;
+import com.google.common.collect.Lists;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -191,181 +131,195 @@ public class OzoneManagerRequestHandler {
this.impl = om;
}
+ //TODO: use map to make shorted methods
+ @SuppressWarnings("methodlength")
public OMResponse handle(OMRequest request) {
LOG.debug("Received OMRequest: {}, ", request);
Type cmdType = request.getCmdType();
OMResponse.Builder responseBuilder = OMResponse.newBuilder()
- .setCmdType(cmdType);
-
- switch (cmdType) {
- case CreateVolume:
- CreateVolumeResponse createVolumeResponse = createVolume(
- request.getCreateVolumeRequest());
- responseBuilder.setCreateVolumeResponse(createVolumeResponse);
- break;
- case SetVolumeProperty:
- SetVolumePropertyResponse setVolumePropertyResponse = setVolumeProperty(
- request.getSetVolumePropertyRequest());
- responseBuilder.setSetVolumePropertyResponse(setVolumePropertyResponse);
- break;
- case CheckVolumeAccess:
- CheckVolumeAccessResponse checkVolumeAccessResponse = checkVolumeAccess(
- request.getCheckVolumeAccessRequest());
- responseBuilder.setCheckVolumeAccessResponse(checkVolumeAccessResponse);
- break;
- case InfoVolume:
- InfoVolumeResponse infoVolumeResponse = infoVolume(
- request.getInfoVolumeRequest());
- responseBuilder.setInfoVolumeResponse(infoVolumeResponse);
- break;
- case DeleteVolume:
- DeleteVolumeResponse deleteVolumeResponse = deleteVolume(
- request.getDeleteVolumeRequest());
- responseBuilder.setDeleteVolumeResponse(deleteVolumeResponse);
- break;
- case ListVolume:
- ListVolumeResponse listVolumeResponse = listVolumes(
- request.getListVolumeRequest());
- responseBuilder.setListVolumeResponse(listVolumeResponse);
- break;
- case CreateBucket:
- CreateBucketResponse createBucketResponse = createBucket(
- request.getCreateBucketRequest());
- responseBuilder.setCreateBucketResponse(createBucketResponse);
- break;
- case InfoBucket:
- InfoBucketResponse infoBucketResponse = infoBucket(
- request.getInfoBucketRequest());
- responseBuilder.setInfoBucketResponse(infoBucketResponse);
- break;
- case SetBucketProperty:
- SetBucketPropertyResponse setBucketPropertyResponse = setBucketProperty(
- request.getSetBucketPropertyRequest());
- responseBuilder.setSetBucketPropertyResponse(setBucketPropertyResponse);
- break;
- case DeleteBucket:
- DeleteBucketResponse deleteBucketResponse = deleteBucket(
- request.getDeleteBucketRequest());
- responseBuilder.setDeleteBucketResponse(deleteBucketResponse);
- break;
- case ListBuckets:
- ListBucketsResponse listBucketsResponse = listBuckets(
- request.getListBucketsRequest());
- responseBuilder.setListBucketsResponse(listBucketsResponse);
- break;
- case CreateKey:
- CreateKeyResponse createKeyResponse = createKey(
- request.getCreateKeyRequest());
- responseBuilder.setCreateKeyResponse(createKeyResponse);
- break;
- case LookupKey:
- LookupKeyResponse lookupKeyResponse = lookupKey(
- request.getLookupKeyRequest());
- responseBuilder.setLookupKeyResponse(lookupKeyResponse);
- break;
- case RenameKey:
- RenameKeyResponse renameKeyResponse = renameKey(
- request.getRenameKeyRequest());
- responseBuilder.setRenameKeyResponse(renameKeyResponse);
- break;
- case DeleteKey:
- DeleteKeyResponse deleteKeyResponse = deleteKey(
- request.getDeleteKeyRequest());
- responseBuilder.setDeleteKeyResponse(deleteKeyResponse);
- break;
- case ListKeys:
- ListKeysResponse listKeysResponse = listKeys(
- request.getListKeysRequest());
- responseBuilder.setListKeysResponse(listKeysResponse);
- break;
- case CommitKey:
- CommitKeyResponse commitKeyResponse = commitKey(
- request.getCommitKeyRequest());
- responseBuilder.setCommitKeyResponse(commitKeyResponse);
- break;
- case AllocateBlock:
- AllocateBlockResponse allocateBlockResponse = allocateBlock(
- request.getAllocateBlockRequest());
- responseBuilder.setAllocateBlockResponse(allocateBlockResponse);
- break;
- case CreateS3Bucket:
- S3CreateBucketResponse s3CreateBucketResponse = createS3Bucket(
- request.getCreateS3BucketRequest());
- responseBuilder.setCreateS3BucketResponse(s3CreateBucketResponse);
- break;
- case DeleteS3Bucket:
- S3DeleteBucketResponse s3DeleteBucketResponse = deleteS3Bucket(
- request.getDeleteS3BucketRequest());
- responseBuilder.setDeleteS3BucketResponse(s3DeleteBucketResponse);
- break;
- case InfoS3Bucket:
- S3BucketInfoResponse s3BucketInfoResponse = getS3Bucketinfo(
- request.getInfoS3BucketRequest());
- responseBuilder.setInfoS3BucketResponse(s3BucketInfoResponse);
- break;
- case ListS3Buckets:
- S3ListBucketsResponse s3ListBucketsResponse = listS3Buckets(
- request.getListS3BucketsRequest());
- responseBuilder.setListS3BucketsResponse(s3ListBucketsResponse);
- break;
- case InitiateMultiPartUpload:
- MultipartInfoInitiateResponse multipartInfoInitiateResponse =
- initiateMultiPartUpload(request.getInitiateMultiPartUploadRequest());
- responseBuilder.setInitiateMultiPartUploadResponse(
- multipartInfoInitiateResponse);
- break;
- case CommitMultiPartUpload:
- MultipartCommitUploadPartResponse commitUploadPartResponse =
- commitMultipartUploadPart(request.getCommitMultiPartUploadRequest());
- responseBuilder.setCommitMultiPartUploadResponse(
- commitUploadPartResponse);
- break;
- case CompleteMultiPartUpload:
- MultipartUploadCompleteResponse completeMultiPartUploadResponse =
- completeMultipartUpload(request.getCompleteMultiPartUploadRequest());
- responseBuilder.setCompleteMultiPartUploadResponse(
- completeMultiPartUploadResponse);
- break;
- case AbortMultiPartUpload:
- MultipartUploadAbortResponse abortMultiPartAbortResponse =
- abortMultipartUpload(request.getAbortMultiPartUploadRequest());
- responseBuilder.setAbortMultiPartUploadResponse(
- abortMultiPartAbortResponse);
- break;
- case ListMultiPartUploadParts:
- MultipartUploadListPartsResponse listPartsResponse =
- listParts(request.getListMultipartUploadPartsRequest());
- responseBuilder.setListMultipartUploadPartsResponse(listPartsResponse);
- break;
- case ServiceList:
- ServiceListResponse serviceListResponse = getServiceList(
- request.getServiceListRequest());
- responseBuilder.setServiceListResponse(serviceListResponse);
- break;
- case GetDelegationToken:
- GetDelegationTokenResponseProto getDtResp = getDelegationToken(
- request.getGetDelegationTokenRequest());
- responseBuilder.setGetDelegationTokenResponse(getDtResp);
- break;
- case RenewDelegationToken:
- RenewDelegationTokenResponseProto renewDtResp = renewDelegationToken(
- request.getRenewDelegationTokenRequest());
- responseBuilder.setRenewDelegationTokenResponse(renewDtResp);
- break;
- case CancelDelegationToken:
- CancelDelegationTokenResponseProto cancelDtResp = cancelDelegationToken(
- request.getCancelDelegationTokenRequest());
- responseBuilder.setCancelDelegationTokenResponse(cancelDtResp);
- break;
- case GetS3Secret:
- GetS3SecretResponse getS3SecretResp = getS3Secret(request
- .getGetS3SecretRequest());
- responseBuilder.setGetS3SecretResponse(getS3SecretResp);
- break;
- default:
+ .setCmdType(cmdType)
+ .setStatus(Status.OK);
+ try {
+ switch (cmdType) {
+ case CreateVolume:
+ CreateVolumeResponse createVolumeResponse = createVolume(
+ request.getCreateVolumeRequest());
+ responseBuilder.setCreateVolumeResponse(createVolumeResponse);
+ break;
+ case SetVolumeProperty:
+ SetVolumePropertyResponse setVolumePropertyResponse = setVolumeProperty(
+ request.getSetVolumePropertyRequest());
+ responseBuilder.setSetVolumePropertyResponse(setVolumePropertyResponse);
+ break;
+ case CheckVolumeAccess:
+ CheckVolumeAccessResponse checkVolumeAccessResponse = checkVolumeAccess(
+ request.getCheckVolumeAccessRequest());
+ responseBuilder.setCheckVolumeAccessResponse(checkVolumeAccessResponse);
+ break;
+ case InfoVolume:
+ InfoVolumeResponse infoVolumeResponse = infoVolume(
+ request.getInfoVolumeRequest());
+ responseBuilder.setInfoVolumeResponse(infoVolumeResponse);
+ break;
+ case DeleteVolume:
+ DeleteVolumeResponse deleteVolumeResponse = deleteVolume(
+ request.getDeleteVolumeRequest());
+ responseBuilder.setDeleteVolumeResponse(deleteVolumeResponse);
+ break;
+ case ListVolume:
+ ListVolumeResponse listVolumeResponse = listVolumes(
+ request.getListVolumeRequest());
+ responseBuilder.setListVolumeResponse(listVolumeResponse);
+ break;
+ case CreateBucket:
+ CreateBucketResponse createBucketResponse = createBucket(
+ request.getCreateBucketRequest());
+ responseBuilder.setCreateBucketResponse(createBucketResponse);
+ break;
+ case InfoBucket:
+ InfoBucketResponse infoBucketResponse = infoBucket(
+ request.getInfoBucketRequest());
+ responseBuilder.setInfoBucketResponse(infoBucketResponse);
+ break;
+ case SetBucketProperty:
+ SetBucketPropertyResponse setBucketPropertyResponse = setBucketProperty(
+ request.getSetBucketPropertyRequest());
+ responseBuilder.setSetBucketPropertyResponse(setBucketPropertyResponse);
+ break;
+ case DeleteBucket:
+ DeleteBucketResponse deleteBucketResponse = deleteBucket(
+ request.getDeleteBucketRequest());
+ responseBuilder.setDeleteBucketResponse(deleteBucketResponse);
+ break;
+ case ListBuckets:
+ ListBucketsResponse listBucketsResponse = listBuckets(
+ request.getListBucketsRequest());
+ responseBuilder.setListBucketsResponse(listBucketsResponse);
+ break;
+ case CreateKey:
+ CreateKeyResponse createKeyResponse = createKey(
+ request.getCreateKeyRequest());
+ responseBuilder.setCreateKeyResponse(createKeyResponse);
+ break;
+ case LookupKey:
+ LookupKeyResponse lookupKeyResponse = lookupKey(
+ request.getLookupKeyRequest());
+ responseBuilder.setLookupKeyResponse(lookupKeyResponse);
+ break;
+ case RenameKey:
+ RenameKeyResponse renameKeyResponse = renameKey(
+ request.getRenameKeyRequest());
+ responseBuilder.setRenameKeyResponse(renameKeyResponse);
+ break;
+ case DeleteKey:
+ DeleteKeyResponse deleteKeyResponse = deleteKey(
+ request.getDeleteKeyRequest());
+ responseBuilder.setDeleteKeyResponse(deleteKeyResponse);
+ break;
+ case ListKeys:
+ ListKeysResponse listKeysResponse = listKeys(
+ request.getListKeysRequest());
+ responseBuilder.setListKeysResponse(listKeysResponse);
+ break;
+ case CommitKey:
+ CommitKeyResponse commitKeyResponse = commitKey(
+ request.getCommitKeyRequest());
+ responseBuilder.setCommitKeyResponse(commitKeyResponse);
+ break;
+ case AllocateBlock:
+ AllocateBlockResponse allocateBlockResponse = allocateBlock(
+ request.getAllocateBlockRequest());
+ responseBuilder.setAllocateBlockResponse(allocateBlockResponse);
+ break;
+ case CreateS3Bucket:
+ S3CreateBucketResponse s3CreateBucketResponse = createS3Bucket(
+ request.getCreateS3BucketRequest());
+ responseBuilder.setCreateS3BucketResponse(s3CreateBucketResponse);
+ break;
+ case DeleteS3Bucket:
+ S3DeleteBucketResponse s3DeleteBucketResponse = deleteS3Bucket(
+ request.getDeleteS3BucketRequest());
+ responseBuilder.setDeleteS3BucketResponse(s3DeleteBucketResponse);
+ break;
+ case InfoS3Bucket:
+ S3BucketInfoResponse s3BucketInfoResponse = getS3Bucketinfo(
+ request.getInfoS3BucketRequest());
+ responseBuilder.setInfoS3BucketResponse(s3BucketInfoResponse);
+ break;
+ case ListS3Buckets:
+ S3ListBucketsResponse s3ListBucketsResponse = listS3Buckets(
+ request.getListS3BucketsRequest());
+ responseBuilder.setListS3BucketsResponse(s3ListBucketsResponse);
+ break;
+ case InitiateMultiPartUpload:
+ MultipartInfoInitiateResponse multipartInfoInitiateResponse =
+ initiateMultiPartUpload(
+ request.getInitiateMultiPartUploadRequest());
+ responseBuilder.setInitiateMultiPartUploadResponse(
+ multipartInfoInitiateResponse);
+ break;
+ case CommitMultiPartUpload:
+ MultipartCommitUploadPartResponse commitUploadPartResponse =
+ commitMultipartUploadPart(
+ request.getCommitMultiPartUploadRequest());
+ responseBuilder.setCommitMultiPartUploadResponse(
+ commitUploadPartResponse);
+ break;
+ case CompleteMultiPartUpload:
+ MultipartUploadCompleteResponse completeMultiPartUploadResponse =
+ completeMultipartUpload(
+ request.getCompleteMultiPartUploadRequest());
+ responseBuilder.setCompleteMultiPartUploadResponse(
+ completeMultiPartUploadResponse);
+ break;
+ case AbortMultiPartUpload:
+ MultipartUploadAbortResponse abortMultiPartAbortResponse =
+ abortMultipartUpload(request.getAbortMultiPartUploadRequest());
+ responseBuilder.setAbortMultiPartUploadResponse(
+ abortMultiPartAbortResponse);
+ break;
+ case ListMultiPartUploadParts:
+ MultipartUploadListPartsResponse listPartsResponse =
+ listParts(request.getListMultipartUploadPartsRequest());
+ responseBuilder.setListMultipartUploadPartsResponse(listPartsResponse);
+ break;
+ case ServiceList:
+ ServiceListResponse serviceListResponse = getServiceList(
+ request.getServiceListRequest());
+ responseBuilder.setServiceListResponse(serviceListResponse);
+ break;
+ case GetDelegationToken:
+ GetDelegationTokenResponseProto getDtResp = getDelegationToken(
+ request.getGetDelegationTokenRequest());
+ responseBuilder.setGetDelegationTokenResponse(getDtResp);
+ break;
+ case RenewDelegationToken:
+ RenewDelegationTokenResponseProto renewDtResp = renewDelegationToken(
+ request.getRenewDelegationTokenRequest());
+ responseBuilder.setRenewDelegationTokenResponse(renewDtResp);
+ break;
+ case CancelDelegationToken:
+ CancelDelegationTokenResponseProto cancelDtResp = cancelDelegationToken(
+ request.getCancelDelegationTokenRequest());
+ responseBuilder.setCancelDelegationTokenResponse(cancelDtResp);
+ break;
+ case GetS3Secret:
+ GetS3SecretResponse getS3SecretResp = getS3Secret(request
+ .getGetS3SecretRequest());
+ responseBuilder.setGetS3SecretResponse(getS3SecretResp);
+ break;
+ default:
+ responseBuilder.setSuccess(false);
+ responseBuilder.setMessage("Unrecognized Command Type: " + cmdType);
+ break;
+ }
+ responseBuilder.setSuccess(true);
+ } catch (IOException ex) {
responseBuilder.setSuccess(false);
- responseBuilder.setMessage("Unrecognized Command Type: " + cmdType);
- break;
+ responseBuilder.setStatus(exceptionToResponseStatus(ex));
+ if (ex.getMessage() != null) {
+ responseBuilder.setMessage(ex.getMessage());
+ }
}
return responseBuilder.build();
}
@@ -373,75 +327,7 @@ public class OzoneManagerRequestHandler {
// Convert and exception to corresponding status code
private Status exceptionToResponseStatus(IOException ex) {
if (ex instanceof OMException) {
- OMException omException = (OMException)ex;
- switch (omException.getResult()) {
- case FAILED_VOLUME_ALREADY_EXISTS:
- return Status.VOLUME_ALREADY_EXISTS;
- case FAILED_TOO_MANY_USER_VOLUMES:
- return Status.USER_TOO_MANY_VOLUMES;
- case FAILED_VOLUME_NOT_FOUND:
- return Status.VOLUME_NOT_FOUND;
- case FAILED_VOLUME_NOT_EMPTY:
- return Status.VOLUME_NOT_EMPTY;
- case FAILED_USER_NOT_FOUND:
- return Status.USER_NOT_FOUND;
- case FAILED_BUCKET_ALREADY_EXISTS:
- return Status.BUCKET_ALREADY_EXISTS;
- case FAILED_BUCKET_NOT_FOUND:
- return Status.BUCKET_NOT_FOUND;
- case FAILED_BUCKET_NOT_EMPTY:
- return Status.BUCKET_NOT_EMPTY;
- case FAILED_KEY_ALREADY_EXISTS:
- return Status.KEY_ALREADY_EXISTS;
- case FAILED_KEY_NOT_FOUND:
- return Status.KEY_NOT_FOUND;
- case FAILED_INVALID_KEY_NAME:
- return Status.INVALID_KEY_NAME;
- case FAILED_KEY_ALLOCATION:
- return Status.KEY_ALLOCATION_ERROR;
- case FAILED_KEY_DELETION:
- return Status.KEY_DELETION_ERROR;
- case FAILED_KEY_RENAME:
- return Status.KEY_RENAME_ERROR;
- case FAILED_METADATA_ERROR:
- return Status.METADATA_ERROR;
- case OM_NOT_INITIALIZED:
- return Status.OM_NOT_INITIALIZED;
- case SCM_VERSION_MISMATCH_ERROR:
- return Status.SCM_VERSION_MISMATCH_ERROR;
- case S3_BUCKET_ALREADY_EXISTS:
- return Status.S3_BUCKET_ALREADY_EXISTS;
- case S3_BUCKET_NOT_FOUND:
- return Status.S3_BUCKET_NOT_FOUND;
- case INITIATE_MULTIPART_UPLOAD_FAILED:
- return Status.INITIATE_MULTIPART_UPLOAD_ERROR;
- case NO_SUCH_MULTIPART_UPLOAD:
- return Status.NO_SUCH_MULTIPART_UPLOAD_ERROR;
- case UPLOAD_PART_FAILED:
- return Status.MULTIPART_UPLOAD_PARTFILE_ERROR;
- case COMPLETE_MULTIPART_UPLOAD_FAILED:
- return Status.COMPLETE_MULTIPART_UPLOAD_ERROR;
- case MISMATCH_MULTIPART_LIST:
- return Status.MISMATCH_MULTIPART_LIST;
- case MISSING_UPLOAD_PARTS:
- return Status.MISSING_UPLOAD_PARTS;
- case ENTITY_TOO_SMALL:
- return Status.ENTITY_TOO_SMALL;
- case ABORT_MULTIPART_UPLOAD_FAILED:
- return Status.ABORT_MULTIPART_UPLOAD_FAILED;
- case LIST_MULTIPART_UPLOAD_PARTS_FAILED:
- return Status.LIST_MULTIPART_UPLOAD_PARTS_FAILED;
- case INVALID_AUTH_METHOD:
- return Status.INVALID_AUTH_METHOD;
- case INVALID_TOKEN:
- return Status.INVALID_TOKEN;
- case TOKEN_EXPIRED:
- return Status.TOKEN_EXPIRED;
- case TOKEN_ERROR_OTHER:
- return Status.TOKEN_ERROR_OTHER;
- default:
- return Status.INTERNAL_ERROR;
- }
+ return Status.values()[((OMException) ex).getResult().ordinal()];
} else {
if (LOG.isDebugEnabled()) {
LOG.debug("Unknown error occurs", ex);
@@ -453,6 +339,7 @@ public class OzoneManagerRequestHandler {
/**
* Validates that the incoming OM request has required parameters.
* TODO: Add more validation checks before writing the request to Ratis log.
+ *
* @param omRequest client request to OM
* @throws OMException thrown if required parameters are set to null.
*/
@@ -468,614 +355,525 @@ public class OzoneManagerRequestHandler {
}
}
- private CreateVolumeResponse createVolume(CreateVolumeRequest request) {
- CreateVolumeResponse.Builder resp = CreateVolumeResponse.newBuilder();
- resp.setStatus(Status.OK);
- try {
- impl.createVolume(OmVolumeArgs.getFromProtobuf(request.getVolumeInfo()));
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
- return resp.build();
+ private CreateVolumeResponse createVolume(CreateVolumeRequest request)
+ throws IOException {
+ impl.createVolume(OmVolumeArgs.getFromProtobuf(request.getVolumeInfo()));
+ return
+ CreateVolumeResponse.newBuilder().build();
}
private SetVolumePropertyResponse setVolumeProperty(
- SetVolumePropertyRequest request) {
+ SetVolumePropertyRequest request) throws IOException {
SetVolumePropertyResponse.Builder resp =
SetVolumePropertyResponse.newBuilder();
- resp.setStatus(Status.OK);
+
String volume = request.getVolumeName();
- try {
- if (request.hasQuotaInBytes()) {
- long quota = request.getQuotaInBytes();
- impl.setQuota(volume, quota);
- } else {
- String owner = request.getOwnerName();
- impl.setOwner(volume, owner);
- }
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
+ if (request.hasQuotaInBytes()) {
+ long quota = request.getQuotaInBytes();
+ impl.setQuota(volume, quota);
+ } else {
+ String owner = request.getOwnerName();
+ impl.setOwner(volume, owner);
}
+
return resp.build();
}
private CheckVolumeAccessResponse checkVolumeAccess(
- CheckVolumeAccessRequest request) {
+ CheckVolumeAccessRequest request) throws IOException {
CheckVolumeAccessResponse.Builder resp =
CheckVolumeAccessResponse.newBuilder();
- resp.setStatus(Status.OK);
- try {
- boolean access = impl.checkVolumeAccess(request.getVolumeName(),
- request.getUserAcl());
- // if no access, set the response status as access denied
- if (!access) {
- resp.setStatus(Status.ACCESS_DENIED);
- }
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
+ boolean access = impl.checkVolumeAccess(request.getVolumeName(),
+ request.getUserAcl());
+ // if no access, set the response status as access denied
+
+ if (!access) {
+ throw new OMException(OMException.ResultCodes.ACCESS_DENIED);
}
return resp.build();
}
- private InfoVolumeResponse infoVolume(InfoVolumeRequest request) {
+ private InfoVolumeResponse infoVolume(InfoVolumeRequest request)
+ throws IOException {
InfoVolumeResponse.Builder resp = InfoVolumeResponse.newBuilder();
- resp.setStatus(Status.OK);
String volume = request.getVolumeName();
- try {
- OmVolumeArgs ret = impl.getVolumeInfo(volume);
- resp.setVolumeInfo(ret.getProtobuf());
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ OmVolumeArgs ret = impl.getVolumeInfo(volume);
+ resp.setVolumeInfo(ret.getProtobuf());
+
return resp.build();
}
- private DeleteVolumeResponse deleteVolume(DeleteVolumeRequest request) {
+ private DeleteVolumeResponse deleteVolume(DeleteVolumeRequest request)
+ throws IOException {
DeleteVolumeResponse.Builder resp = DeleteVolumeResponse.newBuilder();
- resp.setStatus(Status.OK);
- try {
- impl.deleteVolume(request.getVolumeName());
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ impl.deleteVolume(request.getVolumeName());
+
return resp.build();
}
- private ListVolumeResponse listVolumes(ListVolumeRequest request) {
+ private ListVolumeResponse listVolumes(ListVolumeRequest request)
+ throws IOException {
ListVolumeResponse.Builder resp = ListVolumeResponse.newBuilder();
List<OmVolumeArgs> result = Lists.newArrayList();
- try {
- if (request.getScope()
- == ListVolumeRequest.Scope.VOLUMES_BY_USER) {
- result = impl.listVolumeByUser(request.getUserName(),
- request.getPrefix(), request.getPrevKey(), request.getMaxKeys());
- } else if (request.getScope()
- == ListVolumeRequest.Scope.VOLUMES_BY_CLUSTER) {
- result = impl.listAllVolumes(request.getPrefix(), request.getPrevKey(),
- request.getMaxKeys());
- }
- result.forEach(item -> resp.addVolumeInfo(item.getProtobuf()));
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
+ if (request.getScope()
+ == ListVolumeRequest.Scope.VOLUMES_BY_USER) {
+ result = impl.listVolumeByUser(request.getUserName(),
+ request.getPrefix(), request.getPrevKey(), request.getMaxKeys());
+ } else if (request.getScope()
+ == ListVolumeRequest.Scope.VOLUMES_BY_CLUSTER) {
+ result =
+ impl.listAllVolumes(request.getPrefix(), request.getPrevKey(),
+ request.getMaxKeys());
}
+
+ result.forEach(item -> resp.addVolumeInfo(item.getProtobuf()));
+
return resp.build();
}
- private CreateBucketResponse createBucket(CreateBucketRequest request) {
+ private CreateBucketResponse createBucket(CreateBucketRequest request)
+ throws IOException {
CreateBucketResponse.Builder resp =
CreateBucketResponse.newBuilder();
- try {
- impl.createBucket(OmBucketInfo.getFromProtobuf(
- request.getBucketInfo()));
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+ impl.createBucket(OmBucketInfo.getFromProtobuf(
+ request.getBucketInfo()));
return resp.build();
}
- private InfoBucketResponse infoBucket(InfoBucketRequest request) {
+ private InfoBucketResponse infoBucket(InfoBucketRequest request)
+ throws IOException {
InfoBucketResponse.Builder resp =
InfoBucketResponse.newBuilder();
- try {
- OmBucketInfo omBucketInfo = impl.getBucketInfo(
- request.getVolumeName(), request.getBucketName());
- resp.setStatus(Status.OK);
- resp.setBucketInfo(omBucketInfo.getProtobuf());
- } catch(IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+ OmBucketInfo omBucketInfo = impl.getBucketInfo(
+ request.getVolumeName(), request.getBucketName());
+ resp.setBucketInfo(omBucketInfo.getProtobuf());
+
return resp.build();
}
- private CreateKeyResponse createKey(CreateKeyRequest request) {
+ private CreateKeyResponse createKey(CreateKeyRequest request)
+ throws IOException {
CreateKeyResponse.Builder resp =
CreateKeyResponse.newBuilder();
- try {
- KeyArgs keyArgs = request.getKeyArgs();
- HddsProtos.ReplicationType type =
- keyArgs.hasType()? keyArgs.getType() : null;
- HddsProtos.ReplicationFactor factor =
- keyArgs.hasFactor()? keyArgs.getFactor() : null;
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .setDataSize(keyArgs.getDataSize())
- .setType(type)
- .setFactor(factor)
- .setIsMultipartKey(keyArgs.getIsMultipartKey())
- .setMultipartUploadID(keyArgs.getMultipartUploadID())
- .setMultipartUploadPartNumber(keyArgs.getMultipartNumber())
- .build();
- if (keyArgs.hasDataSize()) {
- omKeyArgs.setDataSize(keyArgs.getDataSize());
- } else {
- omKeyArgs.setDataSize(0);
- }
- OpenKeySession openKey = impl.openKey(omKeyArgs);
- resp.setKeyInfo(openKey.getKeyInfo().getProtobuf());
- resp.setID(openKey.getId());
- resp.setOpenVersion(openKey.getOpenVersion());
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
+ KeyArgs keyArgs = request.getKeyArgs();
+ HddsProtos.ReplicationType type =
+ keyArgs.hasType() ? keyArgs.getType() : null;
+ HddsProtos.ReplicationFactor factor =
+ keyArgs.hasFactor() ? keyArgs.getFactor() : null;
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .setDataSize(keyArgs.getDataSize())
+ .setType(type)
+ .setFactor(factor)
+ .setIsMultipartKey(keyArgs.getIsMultipartKey())
+ .setMultipartUploadID(keyArgs.getMultipartUploadID())
+ .setMultipartUploadPartNumber(keyArgs.getMultipartNumber())
+ .build();
+ if (keyArgs.hasDataSize()) {
+ omKeyArgs.setDataSize(keyArgs.getDataSize());
+ } else {
+ omKeyArgs.setDataSize(0);
}
+ OpenKeySession openKey = impl.openKey(omKeyArgs);
+ resp.setKeyInfo(openKey.getKeyInfo().getProtobuf());
+ resp.setID(openKey.getId());
+ resp.setOpenVersion(openKey.getOpenVersion());
return resp.build();
}
- private LookupKeyResponse lookupKey(LookupKeyRequest request) {
+ private LookupKeyResponse lookupKey(LookupKeyRequest request)
+ throws IOException {
LookupKeyResponse.Builder resp =
LookupKeyResponse.newBuilder();
- try {
- KeyArgs keyArgs = request.getKeyArgs();
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .build();
- OmKeyInfo keyInfo = impl.lookupKey(omKeyArgs);
- resp.setKeyInfo(keyInfo.getProtobuf());
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+ KeyArgs keyArgs = request.getKeyArgs();
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .build();
+ OmKeyInfo keyInfo = impl.lookupKey(omKeyArgs);
+ resp.setKeyInfo(keyInfo.getProtobuf());
+
return resp.build();
}
- private RenameKeyResponse renameKey(RenameKeyRequest request) {
+ private RenameKeyResponse renameKey(RenameKeyRequest request)
+ throws IOException {
RenameKeyResponse.Builder resp = RenameKeyResponse.newBuilder();
- try {
- KeyArgs keyArgs = request.getKeyArgs();
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .build();
- impl.renameKey(omKeyArgs, request.getToKeyName());
- resp.setStatus(Status.OK);
- } catch (IOException e){
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ KeyArgs keyArgs = request.getKeyArgs();
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .build();
+ impl.renameKey(omKeyArgs, request.getToKeyName());
+
return resp.build();
}
private SetBucketPropertyResponse setBucketProperty(
- SetBucketPropertyRequest request) {
+ SetBucketPropertyRequest request) throws IOException {
SetBucketPropertyResponse.Builder resp =
SetBucketPropertyResponse.newBuilder();
- try {
- impl.setBucketProperty(OmBucketArgs.getFromProtobuf(
- request.getBucketArgs()));
- resp.setStatus(Status.OK);
- } catch(IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+ impl.setBucketProperty(OmBucketArgs.getFromProtobuf(
+ request.getBucketArgs()));
+
return resp.build();
}
- private DeleteKeyResponse deleteKey(DeleteKeyRequest request) {
+ private DeleteKeyResponse deleteKey(DeleteKeyRequest request)
+ throws IOException {
DeleteKeyResponse.Builder resp =
DeleteKeyResponse.newBuilder();
- try {
- KeyArgs keyArgs = request.getKeyArgs();
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .build();
- impl.deleteKey(omKeyArgs);
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ KeyArgs keyArgs = request.getKeyArgs();
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .build();
+ impl.deleteKey(omKeyArgs);
+
return resp.build();
}
- private DeleteBucketResponse deleteBucket(DeleteBucketRequest request) {
+ private DeleteBucketResponse deleteBucket(DeleteBucketRequest request)
+ throws IOException {
DeleteBucketResponse.Builder resp = DeleteBucketResponse.newBuilder();
- resp.setStatus(Status.OK);
- try {
- impl.deleteBucket(request.getVolumeName(), request.getBucketName());
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ impl.deleteBucket(request.getVolumeName(), request.getBucketName());
+
return resp.build();
}
- private ListBucketsResponse listBuckets(ListBucketsRequest request) {
+ private ListBucketsResponse listBuckets(ListBucketsRequest request)
+ throws IOException {
ListBucketsResponse.Builder resp =
ListBucketsResponse.newBuilder();
- try {
- List<OmBucketInfo> buckets = impl.listBuckets(
- request.getVolumeName(),
- request.getStartKey(),
- request.getPrefix(),
- request.getCount());
- for(OmBucketInfo bucket : buckets) {
- resp.addBucketInfo(bucket.getProtobuf());
- }
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
+
+ List<OmBucketInfo> buckets = impl.listBuckets(
+ request.getVolumeName(),
+ request.getStartKey(),
+ request.getPrefix(),
+ request.getCount());
+ for (OmBucketInfo bucket : buckets) {
+ resp.addBucketInfo(bucket.getProtobuf());
}
+
return resp.build();
}
- private ListKeysResponse listKeys(ListKeysRequest request) {
+ private ListKeysResponse listKeys(ListKeysRequest request)
+ throws IOException {
ListKeysResponse.Builder resp =
ListKeysResponse.newBuilder();
- try {
- List<OmKeyInfo> keys = impl.listKeys(
- request.getVolumeName(),
- request.getBucketName(),
- request.getStartKey(),
- request.getPrefix(),
- request.getCount());
- for(OmKeyInfo key : keys) {
- resp.addKeyInfo(key.getProtobuf());
- }
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
+
+ List<OmKeyInfo> keys = impl.listKeys(
+ request.getVolumeName(),
+ request.getBucketName(),
+ request.getStartKey(),
+ request.getPrefix(),
+ request.getCount());
+ for (OmKeyInfo key : keys) {
+ resp.addKeyInfo(key.getProtobuf());
}
+
return resp.build();
}
- private CommitKeyResponse commitKey(CommitKeyRequest request) {
+ private CommitKeyResponse commitKey(CommitKeyRequest request)
+ throws IOException {
CommitKeyResponse.Builder resp =
CommitKeyResponse.newBuilder();
- try {
- KeyArgs keyArgs = request.getKeyArgs();
- HddsProtos.ReplicationType type =
- keyArgs.hasType()? keyArgs.getType() : null;
- HddsProtos.ReplicationFactor factor =
- keyArgs.hasFactor()? keyArgs.getFactor() : null;
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .setLocationInfoList(keyArgs.getKeyLocationsList().stream()
- .map(OmKeyLocationInfo::getFromProtobuf)
- .collect(Collectors.toList()))
- .setType(type)
- .setFactor(factor)
- .setDataSize(keyArgs.getDataSize())
- .build();
- impl.commitKey(omKeyArgs, request.getClientID());
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ KeyArgs keyArgs = request.getKeyArgs();
+ HddsProtos.ReplicationType type =
+ keyArgs.hasType() ? keyArgs.getType() : null;
+ HddsProtos.ReplicationFactor factor =
+ keyArgs.hasFactor() ? keyArgs.getFactor() : null;
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .setLocationInfoList(keyArgs.getKeyLocationsList().stream()
+ .map(OmKeyLocationInfo::getFromProtobuf)
+ .collect(Collectors.toList()))
+ .setType(type)
+ .setFactor(factor)
+ .setDataSize(keyArgs.getDataSize())
+ .build();
+ impl.commitKey(omKeyArgs, request.getClientID());
+
return resp.build();
}
- private AllocateBlockResponse allocateBlock(AllocateBlockRequest request) {
+ private AllocateBlockResponse allocateBlock(AllocateBlockRequest request)
+ throws IOException {
AllocateBlockResponse.Builder resp =
AllocateBlockResponse.newBuilder();
- try {
- KeyArgs keyArgs = request.getKeyArgs();
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .build();
- OmKeyLocationInfo newLocation = impl.allocateBlock(omKeyArgs,
- request.getClientID());
- resp.setKeyLocation(newLocation.getProtobuf());
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ KeyArgs keyArgs = request.getKeyArgs();
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .build();
+ OmKeyLocationInfo newLocation = impl.allocateBlock(omKeyArgs,
+ request.getClientID());
+ resp.setKeyLocation(newLocation.getProtobuf());
+
return resp.build();
}
- private ServiceListResponse getServiceList(ServiceListRequest request) {
+ private ServiceListResponse getServiceList(ServiceListRequest request)
+ throws IOException {
ServiceListResponse.Builder resp = ServiceListResponse.newBuilder();
- try {
- resp.addAllServiceInfo(impl.getServiceList().stream()
- .map(ServiceInfo::getProtobuf)
- .collect(Collectors.toList()));
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ resp.addAllServiceInfo(impl.getServiceList().stream()
+ .map(ServiceInfo::getProtobuf)
+ .collect(Collectors.toList()));
+
return resp.build();
}
- private S3CreateBucketResponse createS3Bucket(S3CreateBucketRequest request) {
+ private S3CreateBucketResponse createS3Bucket(S3CreateBucketRequest request)
+ throws IOException {
S3CreateBucketResponse.Builder resp = S3CreateBucketResponse.newBuilder();
- try {
- impl.createS3Bucket(request.getUserName(), request.getS3Bucketname());
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ impl.createS3Bucket(request.getUserName(), request.getS3Bucketname());
+
return resp.build();
}
- private S3DeleteBucketResponse deleteS3Bucket(S3DeleteBucketRequest request) {
+ private S3DeleteBucketResponse deleteS3Bucket(S3DeleteBucketRequest request)
+ throws IOException {
S3DeleteBucketResponse.Builder resp = S3DeleteBucketResponse.newBuilder();
- try {
- impl.deleteS3Bucket(request.getS3BucketName());
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ impl.deleteS3Bucket(request.getS3BucketName());
+
return resp.build();
}
- private S3BucketInfoResponse getS3Bucketinfo(S3BucketInfoRequest request) {
+ private S3BucketInfoResponse getS3Bucketinfo(S3BucketInfoRequest request)
+ throws IOException {
S3BucketInfoResponse.Builder resp = S3BucketInfoResponse.newBuilder();
- try {
- resp.setOzoneMapping(
- impl.getOzoneBucketMapping(request.getS3BucketName()));
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
- }
+
+ resp.setOzoneMapping(
+ impl.getOzoneBucketMapping(request.getS3BucketName()));
return resp.build();
}
- private S3ListBucketsResponse listS3Buckets(S3ListBucketsRequest request) {
+ private S3ListBucketsResponse listS3Buckets(S3ListBucketsRequest request)
+ throws IOException {
S3ListBucketsResponse.Builder resp = S3ListBucketsResponse.newBuilder();
- try {
- List<OmBucketInfo> buckets = impl.listS3Buckets(
- request.getUserName(),
- request.getStartKey(),
- request.getPrefix(),
- request.getCount());
- for(OmBucketInfo bucket : buckets) {
- resp.addBucketInfo(bucket.getProtobuf());
- }
- resp.setStatus(Status.OK);
- } catch (IOException e) {
- resp.setStatus(exceptionToResponseStatus(e));
+
+ List<OmBucketInfo> buckets = impl.listS3Buckets(
+ request.getUserName(),
+ request.getStartKey(),
+ request.getPrefix(),
+ request.getCount());
+ for (OmBucketInfo bucket : buckets) {
+ resp.addBucketInfo(bucket.getProtobuf());
}
+
return resp.build();
}
private MultipartInfoInitiateResponse initiateMultiPartUpload(
- MultipartInfoInitiateRequest request) {
+ MultipartInfoInitiateRequest request) throws IOException {
MultipartInfoInitiateResponse.Builder resp = MultipartInfoInitiateResponse
.newBuilder();
- try {
- KeyArgs keyArgs = request.getKeyArgs();
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .setType(keyArgs.getType())
- .setFactor(keyArgs.getFactor())
- .build();
- OmMultipartInfo multipartInfo = impl.initiateMultipartUpload(omKeyArgs);
- resp.setVolumeName(multipartInfo.getVolumeName());
- resp.setBucketName(multipartInfo.getBucketName());
- resp.setKeyName(multipartInfo.getKeyName());
- resp.setMultipartUploadID(multipartInfo.getUploadID());
- resp.setStatus(Status.OK);
- } catch (IOException ex) {
- resp.setStatus(exceptionToResponseStatus(ex));
- }
+
+ KeyArgs keyArgs = request.getKeyArgs();
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .setType(keyArgs.getType())
+ .setFactor(keyArgs.getFactor())
+ .build();
+ OmMultipartInfo multipartInfo = impl.initiateMultipartUpload(omKeyArgs);
+ resp.setVolumeName(multipartInfo.getVolumeName());
+ resp.setBucketName(multipartInfo.getBucketName());
+ resp.setKeyName(multipartInfo.getKeyName());
+ resp.setMultipartUploadID(multipartInfo.getUploadID());
+
return resp.build();
}
private MultipartCommitUploadPartResponse commitMultipartUploadPart(
- MultipartCommitUploadPartRequest request) {
+ MultipartCommitUploadPartRequest request) throws IOException {
MultipartCommitUploadPartResponse.Builder resp =
MultipartCommitUploadPartResponse.newBuilder();
- try {
- KeyArgs keyArgs = request.getKeyArgs();
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .setMultipartUploadID(keyArgs.getMultipartUploadID())
- .setIsMultipartKey(keyArgs.getIsMultipartKey())
- .setMultipartUploadPartNumber(keyArgs.getMultipartNumber())
- .setDataSize(keyArgs.getDataSize())
- .setLocationInfoList(keyArgs.getKeyLocationsList().stream()
- .map(OmKeyLocationInfo::getFromProtobuf)
- .collect(Collectors.toList()))
- .build();
- OmMultipartCommitUploadPartInfo commitUploadPartInfo =
- impl.commitMultipartUploadPart(omKeyArgs, request.getClientID());
- resp.setPartName(commitUploadPartInfo.getPartName());
- resp.setStatus(Status.OK);
- } catch (IOException ex) {
- resp.setStatus(exceptionToResponseStatus(ex));
- }
+
+ KeyArgs keyArgs = request.getKeyArgs();
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .setMultipartUploadID(keyArgs.getMultipartUploadID())
+ .setIsMultipartKey(keyArgs.getIsMultipartKey())
+ .setMultipartUploadPartNumber(keyArgs.getMultipartNumber())
+ .setDataSize(keyArgs.getDataSize())
+ .setLocationInfoList(keyArgs.getKeyLocationsList().stream()
+ .map(OmKeyLocationInfo::getFromProtobuf)
+ .collect(Collectors.toList()))
+ .build();
+ OmMultipartCommitUploadPartInfo commitUploadPartInfo =
+ impl.commitMultipartUploadPart(omKeyArgs, request.getClientID());
+ resp.setPartName(commitUploadPartInfo.getPartName());
+
return resp.build();
}
-
private MultipartUploadCompleteResponse completeMultipartUpload(
- MultipartUploadCompleteRequest request) {
+ MultipartUploadCompleteRequest request) throws IOException {
MultipartUploadCompleteResponse.Builder response =
MultipartUploadCompleteResponse.newBuilder();
- try {
- KeyArgs keyArgs = request.getKeyArgs();
- List<Part> partsList = request.getPartsListList();
+ KeyArgs keyArgs = request.getKeyArgs();
+ List<Part> partsList = request.getPartsListList();
- TreeMap<Integer, String> partsMap = new TreeMap<>();
- for (Part part : partsList) {
- partsMap.put(part.getPartNumber(), part.getPartName());
- }
-
- OmMultipartUploadList omMultipartUploadList =
- new OmMultipartUploadList(partsMap);
-
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .setMultipartUploadID(keyArgs.getMultipartUploadID())
- .build();
- OmMultipartUploadCompleteInfo omMultipartUploadCompleteInfo = impl
- .completeMultipartUpload(omKeyArgs, omMultipartUploadList);
-
- response.setVolume(omMultipartUploadCompleteInfo.getVolume())
- .setBucket(omMultipartUploadCompleteInfo.getBucket())
- .setKey(omMultipartUploadCompleteInfo.getKey())
- .setHash(omMultipartUploadCompleteInfo.getHash());
- response.setStatus(Status.OK);
- } catch (IOException ex) {
- response.setStatus(exceptionToResponseStatus(ex));
+ TreeMap<Integer, String> partsMap = new TreeMap<>();
+ for (Part part : partsList) {
+ partsMap.put(part.getPartNumber(), part.getPartName());
}
+
+ OmMultipartUploadList omMultipartUploadList =
+ new OmMultipartUploadList(partsMap);
+
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .setMultipartUploadID(keyArgs.getMultipartUploadID())
+ .build();
+ OmMultipartUploadCompleteInfo omMultipartUploadCompleteInfo = impl
+ .completeMultipartUpload(omKeyArgs, omMultipartUploadList);
+
+ response.setVolume(omMultipartUploadCompleteInfo.getVolume())
+ .setBucket(omMultipartUploadCompleteInfo.getBucket())
+ .setKey(omMultipartUploadCompleteInfo.getKey())
+ .setHash(omMultipartUploadCompleteInfo.getHash());
+
return response.build();
}
private MultipartUploadAbortResponse abortMultipartUpload(
- MultipartUploadAbortRequest multipartUploadAbortRequest) {
+ MultipartUploadAbortRequest multipartUploadAbortRequest)
+ throws IOException {
MultipartUploadAbortResponse.Builder response =
MultipartUploadAbortResponse.newBuilder();
- try {
- KeyArgs keyArgs = multipartUploadAbortRequest.getKeyArgs();
- OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
- .setVolumeName(keyArgs.getVolumeName())
- .setBucketName(keyArgs.getBucketName())
- .setKeyName(keyArgs.getKeyName())
- .setMultipartUploadID(keyArgs.getMultipartUploadID())
- .build();
- impl.abortMultipartUpload(omKeyArgs);
- response.setStatus(Status.OK);
- } catch (IOException ex) {
- response.setStatus(exceptionToResponseStatus(ex));
- }
+ KeyArgs keyArgs = multipartUploadAbortRequest.getKeyArgs();
+ OmKeyArgs omKeyArgs = new OmKeyArgs.Builder()
+ .setVolumeName(keyArgs.getVolumeName())
+ .setBucketName(keyArgs.getBucketName())
+ .setKeyName(keyArgs.getKeyName())
+ .setMultipartUploadID(keyArgs.getMultipartUploadID())
+ .build();
+ impl.abortMultipartUpload(omKeyArgs);
+
return response.build();
}
private MultipartUploadListPartsResponse listParts(
- MultipartUploadListPartsRequest multipartUploadListPartsRequest) {
+ MultipartUploadListPartsRequest multipartUploadListPartsRequest)
+ throws IOException {
MultipartUploadListPartsResponse.Builder response =
MultipartUploadListPartsResponse.newBuilder();
- try {
- OmMultipartUploadListParts omMultipartUploadListParts =
- impl.listParts(multipartUploadListPartsRequest.getVolume(),
- multipartUploadListPartsRequest.getBucket(),
- multipartUploadListPartsRequest.getKey(),
- multipartUploadListPartsRequest.getUploadID(),
- multipartUploadListPartsRequest.getPartNumbermarker(),
- multipartUploadListPartsRequest.getMaxParts());
+ OmMultipartUploadListParts omMultipartUploadListParts =
+ impl.listParts(multipartUploadListPartsRequest.getVolume(),
+ multipartUploadListPartsRequest.getBucket(),
+ multipartUploadListPartsRequest.getKey(),
+ multipartUploadListPartsRequest.getUploadID(),
+ multipartUploadListPartsRequest.getPartNumbermarker(),
+ multipartUploadListPartsRequest.getMaxParts());
- List<OmPartInfo> omPartInfoList =
- omMultipartUploadListParts.getPartInfoList();
+ List<OmPartInfo> omPartInfoList =
+ omMultipartUploadListParts.getPartInfoList();
- List<OzoneManagerProtocolProtos.PartInfo> partInfoList =
- new ArrayList<>();
+ List<OzoneManagerProtocolProtos.PartInfo> partInfoList =
+ new ArrayList<>();
- omPartInfoList.forEach(partInfo -> partInfoList.add(partInfo.getProto()));
+ omPartInfoList.forEach(partInfo -> partInfoList.add(partInfo.getProto()));
- response.setType(omMultipartUploadListParts.getReplicationType());
- response.setNextPartNumberMarker(
- omMultipartUploadListParts.getNextPartNumberMarker());
- response.setIsTruncated(omMultipartUploadListParts.isTruncated());
- response.setStatus(Status.OK);
- return response.addAllPartsList(partInfoList).build();
+ response.setType(omMultipartUploadListParts.getReplicationType());
+ response.setNextPartNumberMarker(
+ omMultipartUploadListParts.getNextPartNumberMarker());
+ response.setIsTruncated(omMultipartUploadListParts.isTruncated());
- } catch (IOException ex) {
- response.setStatus(exceptionToResponseStatus(ex));
- }
+ return response.addAllPartsList(partInfoList).build();
- return response.build();
}
private GetDelegationTokenResponseProto getDelegationToken(
- GetDelegationTokenRequestProto request){
+ GetDelegationTokenRequestProto request) throws OMException {
GetDelegationTokenResponseProto.Builder rb =
GetDelegationTokenResponseProto.newBuilder();
- try {
- Token<OzoneTokenIdentifier> token = impl
- .getDelegationToken(new Text(request.getRenewer()));
- if (token != null) {
- rb.setResponse(org.apache.hadoop.security.proto.SecurityProtos
- .GetDelegationTokenResponseProto.newBuilder().setToken(OMPBHelper
- .convertToTokenProto(token)).build());
- }
- rb.setStatus(Status.OK);
- } catch (IOException ex) {
- rb.setStatus(exceptionToResponseStatus(ex));
+
+ Token<OzoneTokenIdentifier> token = impl
+ .getDelegationToken(new Text(request.getRenewer()));
+ if (token != null) {
+ rb.setResponse(org.apache.hadoop.security.proto.SecurityProtos
+ .GetDelegationTokenResponseProto.newBuilder().setToken(OMPBHelper
+ .convertToTokenProto(token)).build());
}
+
return rb.build();
}
private RenewDelegationTokenResponseProto renewDelegationToken(
- RenewDelegationTokenRequestProto request) {
+ RenewDelegationTokenRequestProto request) throws OMException {
RenewDelegationTokenResponseProto.Builder rb =
RenewDelegationTokenResponseProto.newBuilder();
- try {
- if(request.hasToken()) {
- long expiryTime = impl
- .renewDelegationToken(
- OMPBHelper.convertToDelegationToken(request.getToken()));
- rb.setResponse(org.apache.hadoop.security.proto.SecurityProtos
- .RenewDelegationTokenResponseProto.newBuilder()
- .setNewExpiryTime(expiryTime).build());
- }
- rb.setStatus(Status.OK);
- } catch (IOException ex) {
- rb.setStatus(exceptionToResponseStatus(ex));
+
+ if (request.hasToken()) {
+ long expiryTime = impl
+ .renewDelegationToken(
+ OMPBHelper.convertToDelegationToken(request.getToken()));
+ rb.setResponse(org.apache.hadoop.security.proto.SecurityProtos
+ .RenewDelegationTokenResponseProto.newBuilder()
+ .setNewExpiryTime(expiryTime).build());
}
+
return rb.build();
}
private CancelDelegationTokenResponseProto cancelDelegationToken(
- CancelDelegationTokenRequestProto req) {
+ CancelDelegationTokenRequestProto req) throws OMException {
CancelDelegationTokenResponseProto.Builder rb =
CancelDelegationTokenResponseProto.newBuilder();
- try {
- if(req.hasToken()) {
- impl.cancelDelegationToken(
- OMPBHelper.convertToDelegationToken(req.getToken()));
- }
- rb.setResponse(org.apache.hadoop.security.proto.SecurityProtos
- .CancelDelegationTokenResponseProto.getDefaultInstance());
- rb.setStatus(Status.OK);
- } catch (IOException ex) {
- rb.setStatus(exceptionToResponseStatus(ex));
+
+ if (req.hasToken()) {
+ impl.cancelDelegationToken(
+ OMPBHelper.convertToDelegationToken(req.getToken()));
}
+ rb.setResponse(org.apache.hadoop.security.proto.SecurityProtos
+ .CancelDelegationTokenResponseProto.getDefaultInstance());
+
return rb.build();
}
private OzoneManagerProtocolProtos.GetS3SecretResponse getS3Secret(
- OzoneManagerProtocolProtos.GetS3SecretRequest request) {
+ OzoneManagerProtocolProtos.GetS3SecretRequest request)
+ throws IOException {
OzoneManagerProtocolProtos.GetS3SecretResponse.Builder rb =
OzoneManagerProtocolProtos.GetS3SecretResponse.newBuilder();
- try {
- rb.setS3Secret(impl.getS3Secret(request.getKerberosID()).getProtobuf());
- rb.setStatus(Status.OK);
- } catch (IOException ex) {
- rb.setStatus(exceptionToResponseStatus(ex));
- }
+
+ rb.setS3Secret(impl.getS3Secret(request.getKerberosID()).getProtobuf());
+
return rb.build();
}
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
index f16eb8c..c8cb7c7 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/web/ozShell/Shell.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.web.ozShell;
import org.apache.hadoop.hdds.cli.GenericCli;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.tracing.TracingUtil;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
import org.apache.hadoop.ozone.web.ozShell.bucket.BucketCommands;
import org.apache.hadoop.ozone.web.ozShell.keys.KeyCommands;
import org.apache.hadoop.ozone.web.ozShell.s3.S3Commands;
@@ -93,5 +94,21 @@ public class Shell extends GenericCli {
public static void main(String[] argv) throws Exception {
new Shell().run(argv);
}
+
+ @Override
+ protected void printError(Throwable errorArg) {
+ if (errorArg instanceof OMException) {
+ if (isVerbose()) {
+ errorArg.printStackTrace(System.err);
+ } else {
+ OMException omException = (OMException) errorArg;
+ System.err.println(String
+ .format("%s %s", omException.getResult().name(),
+ omException.getMessage()));
+ }
+ } else {
+ super.printError(errorArg);
+ }
+ }
}
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
index ff0c9a5..248c2d1 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestBucketManagerImpl.java
@@ -93,7 +93,7 @@ public class TestBucketManagerImpl {
.build();
bucketManager.createBucket(bucketInfo);
} catch (OMException omEx) {
- Assert.assertEquals(ResultCodes.FAILED_VOLUME_NOT_FOUND,
+ Assert.assertEquals(ResultCodes.VOLUME_NOT_FOUND,
omEx.getResult());
throw omEx;
} finally {
@@ -130,7 +130,7 @@ public class TestBucketManagerImpl {
bucketManager.createBucket(bucketInfo);
bucketManager.createBucket(bucketInfo);
} catch (OMException omEx) {
- Assert.assertEquals(ResultCodes.FAILED_BUCKET_ALREADY_EXISTS,
+ Assert.assertEquals(ResultCodes.BUCKET_ALREADY_EXISTS,
omEx.getResult());
throw omEx;
} finally {
@@ -148,7 +148,7 @@ public class TestBucketManagerImpl {
BucketManager bucketManager = new BucketManagerImpl(metaMgr);
bucketManager.getBucketInfo("sampleVol", "bucketOne");
} catch (OMException omEx) {
- Assert.assertEquals(ResultCodes.FAILED_BUCKET_NOT_FOUND,
+ Assert.assertEquals(ResultCodes.BUCKET_NOT_FOUND,
omEx.getResult());
throw omEx;
} finally {
@@ -336,7 +336,7 @@ public class TestBucketManagerImpl {
try {
bucketManager.getBucketInfo("sampleVol", "bucket_1");
} catch (OMException omEx) {
- Assert.assertEquals(ResultCodes.FAILED_BUCKET_NOT_FOUND,
+ Assert.assertEquals(ResultCodes.BUCKET_NOT_FOUND,
omEx.getResult());
throw omEx;
}
@@ -373,7 +373,7 @@ public class TestBucketManagerImpl {
try {
bucketManager.deleteBucket("sampleVol", "bucketOne");
} catch (OMException omEx) {
- Assert.assertEquals(ResultCodes.FAILED_BUCKET_NOT_EMPTY,
+ Assert.assertEquals(ResultCodes.BUCKET_NOT_EMPTY,
omEx.getResult());
throw omEx;
}
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
index 67e25c3..b9a29a6 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/BucketEndpoint.java
@@ -40,6 +40,8 @@ import java.util.Iterator;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.ozone.client.OzoneBucket;
import org.apache.hadoop.ozone.client.OzoneKey;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.s3.commontypes.KeyMetadata;
import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteRequest.DeleteObject;
import org.apache.hadoop.ozone.s3.endpoint.MultiDeleteResponse.DeletedObject;
@@ -239,15 +241,13 @@ public class BucketEndpoint extends EndpointBase {
try {
deleteS3Bucket(bucketName);
- } catch (IOException ex) {
- if (ex.getMessage().contains("BUCKET_NOT_EMPTY")) {
- OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.BUCKET_NOT_EMPTY) {
+ throw S3ErrorTable.newError(S3ErrorTable
.BUCKET_NOT_EMPTY, bucketName);
- throw os3Exception;
- } else if (ex.getMessage().contains("BUCKET_NOT_FOUND")) {
- OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
+ } else if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
+ throw S3ErrorTable.newError(S3ErrorTable
.NO_SUCH_BUCKET, bucketName);
- throw os3Exception;
} else {
throw ex;
}
@@ -280,8 +280,8 @@ public class BucketEndpoint extends EndpointBase {
if (!request.isQuiet()) {
result.addDeleted(new DeletedObject(keyToDelete.getKey()));
}
- } catch (IOException ex) {
- if (!ex.getMessage().contains("KEY_NOT_FOUND")) {
+ } catch (OMException ex) {
+ if (ex.getResult() != ResultCodes.KEY_NOT_FOUND) {
result.addError(
new Error(keyToDelete.getKey(), "InternalError",
ex.getMessage()));
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 cfa2117..e111551 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
@@ -25,6 +25,8 @@ import java.util.Iterator;
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.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.s3.exception.OS3Exception;
import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
import org.apache.hadoop.ozone.s3.header.AuthenticationHeaderParser;
@@ -57,12 +59,9 @@ public class EndpointBase {
OzoneBucket bucket;
try {
bucket = volume.getBucket(bucketName);
- } catch (IOException ex) {
- LOG.error("Error occurred is {}", ex);
- if (ex.getMessage().contains("NOT_FOUND")) {
- OS3Exception oex =
- S3ErrorTable.newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName);
- throw oex;
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
+ throw S3ErrorTable.newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName);
} else {
throw ex;
}
@@ -76,12 +75,10 @@ public class EndpointBase {
try {
OzoneVolume volume = getVolume(getOzoneVolumeName(bucketName));
bucket = volume.getBucket(bucketName);
- } catch (IOException ex) {
- LOG.error("Error occurred is {}", ex);
- if (ex.getMessage().contains("NOT_FOUND")) {
- OS3Exception oex =
- S3ErrorTable.newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName);
- throw oex;
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND
+ || ex.getResult() == ResultCodes.S3_BUCKET_NOT_FOUND) {
+ throw S3ErrorTable.newError(S3ErrorTable.NO_SUCH_BUCKET, bucketName);
} else {
throw ex;
}
@@ -93,8 +90,8 @@ public class EndpointBase {
OzoneVolume volume = null;
try {
volume = client.getObjectStore().getVolume(volumeName);
- } catch (Exception ex) {
- if (ex.getMessage().contains("NOT_FOUND")) {
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.VOLUME_NOT_FOUND) {
throw new NotFoundException("Volume " + volumeName + " is not found");
} else {
throw ex;
@@ -115,9 +112,8 @@ public class EndpointBase {
IOException {
try {
client.getObjectStore().createS3Bucket(userName, bucketName);
- } catch (IOException ex) {
- LOG.error("createS3Bucket error:", ex);
- if (!ex.getMessage().contains("ALREADY_EXISTS")) {
+ } catch (OMException ex) {
+ if (ex.getResult() != ResultCodes.VOLUME_ALREADY_EXISTS) {
// S3 does not return error for bucket already exists, it just
// returns the location.
throw ex;
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 8353136..b520e7b 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
@@ -53,6 +53,8 @@ import org.apache.hadoop.ozone.client.OzoneKeyDetails;
import org.apache.hadoop.ozone.client.OzoneMultipartUploadPartListParts;
import org.apache.hadoop.ozone.client.io.OzoneInputStream;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
@@ -293,11 +295,10 @@ public class ObjectEndpoint extends EndpointBase {
}
addLastModifiedDate(responseBuilder, keyDetails);
return responseBuilder.build();
- } catch (IOException ex) {
- if (ex.getMessage().contains("NOT_FOUND")) {
- OS3Exception os3Exception = S3ErrorTable.newError(S3ErrorTable
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
+ throw S3ErrorTable.newError(S3ErrorTable
.NO_SUCH_KEY, keyPath);
- throw os3Exception;
} else {
throw ex;
}
@@ -331,9 +332,8 @@ public class ObjectEndpoint extends EndpointBase {
try {
key = getBucket(bucketName).getKey(keyPath);
// TODO: return the specified range bytes of this object.
- } catch (IOException ex) {
- LOG.error("Exception occurred in HeadObject", ex);
- if (ex.getMessage().contains("KEY_NOT_FOUND")) {
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
// Just return 404 with no content
return Response.status(Status.NOT_FOUND).build();
} else {
@@ -364,8 +364,8 @@ public class ObjectEndpoint extends EndpointBase {
try {
OzoneBucket ozoneBucket = getBucket(bucket);
ozoneBucket.abortMultipartUpload(key, uploadId);
- } catch (IOException ex) {
- if (ex.getMessage().contains("NO_SUCH_MULTIPART_UPLOAD")) {
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR) {
throw S3ErrorTable.newError(S3ErrorTable.NO_SUCH_UPLOAD, uploadId);
}
throw ex;
@@ -385,6 +385,7 @@ public class ObjectEndpoint extends EndpointBase {
* for more details.
*/
@DELETE
+ @SuppressWarnings("emptyblock")
public Response delete(
@PathParam("bucket") String bucketName,
@PathParam("path") String keyPath,
@@ -398,15 +399,17 @@ public class ObjectEndpoint extends EndpointBase {
OzoneBucket bucket = getBucket(bucketName);
bucket.getKey(keyPath);
bucket.deleteKey(keyPath);
- } catch (IOException ex) {
- if (ex.getMessage().contains("BUCKET_NOT_FOUND")) {
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
throw S3ErrorTable.newError(S3ErrorTable
.NO_SUCH_BUCKET, bucketName);
- } else if (!ex.getMessage().contains("NOT_FOUND")) {
+ } else if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
+ //NOT_FOUND is not a problem, AWS doesn't throw exception for missing
+ // keys. Just return 204
+ } else {
throw ex;
}
- //NOT_FOUND is not a problem, AWS doesn't throw exception for missing
- // keys. Just return 204.
+
}
return Response
.status(Status.NO_CONTENT)
@@ -518,22 +521,22 @@ public class ObjectEndpoint extends EndpointBase {
completeMultipartUploadResponse.setLocation(bucket);
return Response.status(Status.OK).entity(completeMultipartUploadResponse)
.build();
- } catch (IOException ex) {
+ } catch (OMException ex) {
LOG.error("Error in Complete Multipart Upload Request for bucket: " +
bucket + ", key: " + key, ex);
- if (ex.getMessage().contains("MISMATCH_MULTIPART_LIST")) {
+ if (ex.getResult() == ResultCodes.MISMATCH_MULTIPART_LIST) {
OS3Exception oex =
S3ErrorTable.newError(S3ErrorTable.INVALID_PART, key);
throw oex;
- } else if (ex.getMessage().contains("MISSING_UPLOAD_PARTS")) {
+ } else if (ex.getResult() == ResultCodes.MISSING_UPLOAD_PARTS) {
OS3Exception oex =
S3ErrorTable.newError(S3ErrorTable.INVALID_PART_ORDER, key);
throw oex;
- } else if (ex.getMessage().contains("NO_SUCH_MULTIPART_UPLOAD_ERROR")) {
+ } else if (ex.getResult() == ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR) {
OS3Exception os3Exception = S3ErrorTable.newError(NO_SUCH_UPLOAD,
uploadID);
throw os3Exception;
- } else if (ex.getMessage().contains("ENTITY_TOO_SMALL")) {
+ } else if (ex.getResult() == ResultCodes.ENTITY_TOO_SMALL) {
OS3Exception os3Exception = S3ErrorTable.newError(ENTITY_TOO_SMALL,
key);
throw os3Exception;
@@ -557,11 +560,10 @@ public class ObjectEndpoint extends EndpointBase {
return Response.status(Status.OK).header("ETag",
omMultipartCommitUploadPartInfo.getPartName()).build();
- } catch (IOException ex) {
- if (ex.getMessage().contains("NO_SUCH_MULTIPART_UPLOAD_ERROR")) {
- OS3Exception os3Exception = S3ErrorTable.newError(NO_SUCH_UPLOAD,
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR) {
+ throw S3ErrorTable.newError(NO_SUCH_UPLOAD,
uploadID);
- throw os3Exception;
}
throw ex;
}
@@ -619,11 +621,10 @@ public class ObjectEndpoint extends EndpointBase {
listPartsResponse.addPart(part);
});
- } catch (IOException ex) {
- if (ex.getMessage().contains("NO_SUCH_MULTIPART_UPLOAD_ERROR")) {
- OS3Exception os3Exception = S3ErrorTable.newError(NO_SUCH_UPLOAD,
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR) {
+ throw S3ErrorTable.newError(NO_SUCH_UPLOAD,
uploadID);
- throw os3Exception;
}
throw ex;
}
@@ -717,11 +718,12 @@ public class ObjectEndpoint extends EndpointBase {
copyObjectResponse.setLastModified(Instant.ofEpochMilli(destKeyDetails
.getModificationTime()));
return copyObjectResponse;
- } catch (IOException ex) {
- if (ex.getMessage().contains("KEY_NOT_FOUND")) {
+ } catch (OMException ex) {
+ if (ex.getResult() == ResultCodes.KEY_NOT_FOUND) {
throw S3ErrorTable.newError(S3ErrorTable.NO_SUCH_KEY, sourceKey);
+ } else if (ex.getResult() == ResultCodes.BUCKET_NOT_FOUND) {
+ throw S3ErrorTable.newError(S3ErrorTable.NO_SUCH_BUCKET, sourceBucket);
}
- LOG.error("Exception occurred in PutObject", ex);
throw ex;
} finally {
if (!closed) {
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 6c85b81..4feaca6 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
@@ -20,9 +20,21 @@
package org.apache.hadoop.ozone.client;
import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
import java.util.stream.Collectors;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_ALREADY_EXISTS;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_EMPTY;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.S3_BUCKET_NOT_FOUND;
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
+
/**
* ObjectStore implementation with in-memory state.
*/
@@ -65,7 +77,7 @@ public class ObjectStoreStub extends ObjectStore {
if (volumes.containsKey(volumeName)) {
return volumes.get(volumeName);
} else {
- throw new IOException("VOLUME_NOT_FOUND");
+ throw new OMException("", VOLUME_NOT_FOUND);
}
}
@@ -118,7 +130,7 @@ public class ObjectStoreStub extends ObjectStore {
createVolume(volumeName);
volumes.get(volumeName).createBucket(s3BucketName);
} else {
- throw new IOException("BUCKET_ALREADY_EXISTS");
+ throw new OMException("", BUCKET_ALREADY_EXISTS);
}
if (userBuckets.get(userName) == null) {
@@ -193,17 +205,17 @@ public class ObjectStoreStub extends ObjectStore {
if (bucketEmptyStatus.get(s3BucketName)) {
bucketVolumeMap.remove(s3BucketName);
} else {
- throw new IOException("BUCKET_NOT_EMPTY");
+ throw new OMException("", BUCKET_NOT_EMPTY);
}
} else {
- throw new IOException("BUCKET_NOT_FOUND");
+ throw new OMException("", BUCKET_NOT_FOUND);
}
}
@Override
public String getOzoneBucketMapping(String s3BucketName) throws IOException {
if (bucketVolumeMap.get(s3BucketName) == null) {
- throw new IOException("S3_BUCKET_NOT_FOUND");
+ throw new OMException("", S3_BUCKET_NOT_FOUND);
}
return bucketVolumeMap.get(s3BucketName);
}
@@ -212,7 +224,7 @@ public class ObjectStoreStub extends ObjectStore {
@SuppressWarnings("StringSplitter")
public String getOzoneVolumeName(String s3BucketName) throws IOException {
if (bucketVolumeMap.get(s3BucketName) == null) {
- throw new IOException("S3_BUCKET_NOT_FOUND");
+ throw new OMException("", S3_BUCKET_NOT_FOUND);
}
return bucketVolumeMap.get(s3BucketName).split("/")[0];
}
@@ -221,7 +233,7 @@ public class ObjectStoreStub extends ObjectStore {
@SuppressWarnings("StringSplitter")
public String getOzoneBucketName(String s3BucketName) throws IOException {
if (bucketVolumeMap.get(s3BucketName) == null) {
- throw new IOException("S3_BUCKET_NOT_FOUND");
+ throw new OMException("", BUCKET_NOT_FOUND);
}
return bucketVolumeMap.get(s3BucketName).split("/")[1];
}
diff --git a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
index 2821362..2df3b56 100644
--- a/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
+++ b/hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/client/OzoneBucketStub.java
@@ -39,6 +39,8 @@ 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.OzoneMultipartUploadPartListParts.PartInfo;
+import org.apache.hadoop.ozone.om.exceptions.OMException;
+import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadCompleteInfo;
import org.apache.hadoop.util.Time;
@@ -124,7 +126,7 @@ public class OzoneBucketStub extends OzoneBucket {
if (keyDetails.containsKey(key)) {
return keyDetails.get(key);
} else {
- throw new IOException("Lookup key failed, error:KEY_NOT_FOUND");
+ throw new OMException(ResultCodes.KEY_NOT_FOUND);
}
}
@@ -177,7 +179,7 @@ public class OzoneBucketStub extends OzoneBucket {
throws IOException {
String multipartUploadID = multipartUploadIdMap.get(key);
if (multipartUploadID == null || !multipartUploadID.equals(uploadID)) {
- throw new IOException("NO_SUCH_MULTIPART_UPLOAD_ERROR");
+ throw new OMException(ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
} else {
ByteArrayOutputStream byteArrayOutputStream =
new ByteArrayOutputStream((int) size) {
@@ -203,21 +205,21 @@ public class OzoneBucketStub extends OzoneBucket {
String uploadID, Map<Integer, String> partsMap) throws IOException {
if (multipartUploadIdMap.get(key) == null) {
- throw new IOException("NO_SUCH_MULTIPART_UPLOAD_ERROR");
+ throw new OMException(ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
} else {
final Map<Integer, Part> partsList = partList.get(key);
if (partsMap.size() != partsList.size()) {
- throw new IOException("MISMATCH_MULTIPART_LIST");
+ throw new OMException(ResultCodes.MISMATCH_MULTIPART_LIST);
}
int count = 1;
for (Map.Entry<Integer, String> part: partsMap.entrySet()) {
if (part.getKey() != count) {
- throw new IOException("MISSING_UPLOAD_PARTS");
+ throw new OMException(ResultCodes.MISSING_UPLOAD_PARTS);
} else if (!part.getValue().equals(
partsList.get(part.getKey()).getPartName())) {
- throw new IOException("MISMATCH_MULTIPART_LIST");
+ throw new OMException(ResultCodes.MISMATCH_MULTIPART_LIST);
} else {
count++;
}
@@ -232,7 +234,7 @@ public class OzoneBucketStub extends OzoneBucket {
public void abortMultipartUpload(String keyName, String uploadID) throws
IOException {
if (multipartUploadIdMap.get(keyName) == null) {
- throw new IOException("NO_SUCH_MULTIPART_UPLOAD");
+ throw new OMException(ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
} else {
multipartUploadIdMap.remove(keyName);
}
@@ -242,7 +244,7 @@ public class OzoneBucketStub extends OzoneBucket {
public OzoneMultipartUploadPartListParts listParts(String key,
String uploadID, int partNumberMarker, int maxParts) throws IOException {
if (multipartUploadIdMap.get(key) == null) {
- throw new IOException("NO_SUCH_MULTIPART_UPLOAD");
+ throw new OMException(ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
}
List<PartInfo> partInfoList = new ArrayList<>();
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org