|
@@ -26,6 +26,7 @@ import org.apache.hadoop.ozone.om.exceptions.OMException;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
|
|
|
+import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
|
|
|
import org.apache.hadoop.ozone.om.request.key.OMKeyRequest;
|
|
|
import org.apache.hadoop.ozone.om.response.OMClientResponse;
|
|
|
import org.apache.hadoop.ozone.om.response.s3.multipart
|
|
@@ -44,6 +45,8 @@ import org.apache.hadoop.ozone.security.acl.OzoneObj;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.apache.hadoop.utils.db.cache.CacheKey;
|
|
|
import org.apache.hadoop.utils.db.cache.CacheValue;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.stream.Collectors;
|
|
@@ -56,6 +59,10 @@ import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_L
|
|
|
*/
|
|
|
public class S3MultipartUploadCommitPartRequest extends OMKeyRequest {
|
|
|
|
|
|
+
|
|
|
+ private static final Logger LOG =
|
|
|
+ LoggerFactory.getLogger(S3MultipartUploadCommitPartRequest.class);
|
|
|
+
|
|
|
public S3MultipartUploadCommitPartRequest(OMRequest omRequest) {
|
|
|
super(omRequest);
|
|
|
}
|
|
@@ -74,7 +81,8 @@ public class S3MultipartUploadCommitPartRequest extends OMKeyRequest {
|
|
|
|
|
|
@Override
|
|
|
public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
|
|
|
- long transactionLogIndex) {
|
|
|
+ long transactionLogIndex,
|
|
|
+ OzoneManagerDoubleBufferHelper ozoneManagerDoubleBufferHelper) {
|
|
|
MultipartCommitUploadPartRequest multipartCommitUploadPartRequest =
|
|
|
getOmRequest().getCommitMultiPartUploadRequest();
|
|
|
|
|
@@ -89,13 +97,19 @@ public class S3MultipartUploadCommitPartRequest extends OMKeyRequest {
|
|
|
ozoneManager.getMetrics().incNumCommitMultipartUploadParts();
|
|
|
|
|
|
boolean acquiredLock = false;
|
|
|
- OmMultipartKeyInfo multipartKeyInfo = null;
|
|
|
- OmKeyInfo omKeyInfo = null;
|
|
|
- String openKey = null;
|
|
|
- String multipartKey = null;
|
|
|
- OzoneManagerProtocolProtos.PartKeyInfo oldPartKeyInfo = null;
|
|
|
+
|
|
|
IOException exception = null;
|
|
|
String partName = null;
|
|
|
+ OMResponse.Builder omResponse = OMResponse.newBuilder()
|
|
|
+ .setCmdType(OzoneManagerProtocolProtos.Type.CommitMultiPartUpload)
|
|
|
+ .setStatus(OzoneManagerProtocolProtos.Status.OK)
|
|
|
+ .setSuccess(true);
|
|
|
+ OMClientResponse omClientResponse = null;
|
|
|
+ OzoneManagerProtocolProtos.PartKeyInfo oldPartKeyInfo = null;
|
|
|
+ String openKey = null;
|
|
|
+ OmKeyInfo omKeyInfo = null;
|
|
|
+ String multipartKey = null;
|
|
|
+ OmMultipartKeyInfo multipartKeyInfo = null;
|
|
|
try {
|
|
|
// check Acl
|
|
|
if (ozoneManager.getAclsEnabled()) {
|
|
@@ -111,11 +125,11 @@ public class S3MultipartUploadCommitPartRequest extends OMKeyRequest {
|
|
|
validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
|
|
|
|
|
|
String uploadID = keyArgs.getMultipartUploadID();
|
|
|
- multipartKey = omMetadataManager.getMultipartKey(volumeName, bucketName,
|
|
|
- keyName, uploadID);
|
|
|
+ multipartKey = omMetadataManager.getMultipartKey(volumeName,
|
|
|
+ bucketName, keyName, uploadID);
|
|
|
|
|
|
- multipartKeyInfo = omMetadataManager
|
|
|
- .getMultipartInfoTable().get(multipartKey);
|
|
|
+ multipartKeyInfo =
|
|
|
+ omMetadataManager.getMultipartInfoTable().get(multipartKey);
|
|
|
|
|
|
long clientID = multipartCommitUploadPartRequest.getClientID();
|
|
|
|
|
@@ -124,7 +138,6 @@ public class S3MultipartUploadCommitPartRequest extends OMKeyRequest {
|
|
|
|
|
|
omKeyInfo = omMetadataManager.getOpenKeyTable().get(openKey);
|
|
|
|
|
|
-
|
|
|
if (omKeyInfo == null) {
|
|
|
throw new OMException("Failed to commit Multipart Upload key, as " +
|
|
|
openKey + "entry is not found in the openKey table", KEY_NOT_FOUND);
|
|
@@ -180,9 +193,23 @@ public class S3MultipartUploadCommitPartRequest extends OMKeyRequest {
|
|
|
new CacheValue<>(Optional.absent(), transactionLogIndex));
|
|
|
}
|
|
|
|
|
|
+ omResponse.setCommitMultiPartUploadResponse(
|
|
|
+ MultipartCommitUploadPartResponse.newBuilder().setPartName(partName));
|
|
|
+ omClientResponse = new S3MultipartUploadCommitPartResponse(multipartKey,
|
|
|
+ openKey, keyArgs.getModificationTime(), omKeyInfo, multipartKeyInfo,
|
|
|
+ oldPartKeyInfo, omResponse.build());
|
|
|
+
|
|
|
} catch (IOException ex) {
|
|
|
exception = ex;
|
|
|
+ omClientResponse = new S3MultipartUploadCommitPartResponse(multipartKey,
|
|
|
+ openKey, keyArgs.getModificationTime(), omKeyInfo, multipartKeyInfo,
|
|
|
+ oldPartKeyInfo, createErrorOMResponse(omResponse, exception));
|
|
|
} finally {
|
|
|
+ if (omClientResponse != null) {
|
|
|
+ omClientResponse.setFlushFuture(
|
|
|
+ ozoneManagerDoubleBufferHelper.add(omClientResponse,
|
|
|
+ transactionLogIndex));
|
|
|
+ }
|
|
|
if (acquiredLock) {
|
|
|
omMetadataManager.getLock().releaseLock(BUCKET_LOCK, volumeName,
|
|
|
bucketName);
|
|
@@ -194,24 +221,16 @@ public class S3MultipartUploadCommitPartRequest extends OMKeyRequest {
|
|
|
OMAction.COMMIT_MULTIPART_UPLOAD_PARTKEY, buildKeyArgsAuditMap(keyArgs),
|
|
|
exception, getOmRequest().getUserInfo()));
|
|
|
|
|
|
- OMResponse.Builder omResponse = OMResponse.newBuilder()
|
|
|
- .setCmdType(OzoneManagerProtocolProtos.Type.CommitMultiPartUpload)
|
|
|
- .setStatus(OzoneManagerProtocolProtos.Status.OK)
|
|
|
- .setSuccess(true);
|
|
|
-
|
|
|
if (exception == null) {
|
|
|
- omResponse.setCommitMultiPartUploadResponse(
|
|
|
- MultipartCommitUploadPartResponse.newBuilder().setPartName(partName));
|
|
|
- return new S3MultipartUploadCommitPartResponse(multipartKey, openKey,
|
|
|
- keyArgs.getModificationTime(), omKeyInfo, multipartKeyInfo,
|
|
|
- oldPartKeyInfo, omResponse.build());
|
|
|
+ LOG.debug("MultipartUpload Commit is successfully for Key:{} in " +
|
|
|
+ "Volume/Bucket {}/{}", keyName, volumeName, bucketName);
|
|
|
+
|
|
|
} else {
|
|
|
+ LOG.error("MultipartUpload Commit is failed for Key:{} in " +
|
|
|
+ "Volume/Bucket {}/{}", keyName, volumeName, bucketName, exception);
|
|
|
ozoneManager.getMetrics().incNumCommitMultipartUploadPartFails();
|
|
|
- return new S3MultipartUploadCommitPartResponse(multipartKey, openKey,
|
|
|
- keyArgs.getModificationTime(), omKeyInfo, multipartKeyInfo,
|
|
|
- oldPartKeyInfo, createErrorOMResponse(omResponse, exception));
|
|
|
-
|
|
|
}
|
|
|
+ return omClientResponse;
|
|
|
}
|
|
|
}
|
|
|
|