|
@@ -213,6 +213,8 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
|
|
|
try {
|
|
|
if (args.getIsMultipartKey()) {
|
|
|
+ Preconditions.checkArgument(args.getMultipartUploadPartNumber() > 0,
|
|
|
+ "PartNumber Should be greater than zero");
|
|
|
// When key is multipart upload part key, we should take replication
|
|
|
// type and replication factor from original key which has done
|
|
|
// initiate multipart upload. If we have not found any such, we throw
|
|
@@ -686,8 +688,16 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
keyInfo.setDataSize(omKeyArgs.getDataSize());
|
|
|
keyInfo.updateLocationInfoList(omKeyArgs.getLocationInfoList());
|
|
|
|
|
|
- partName = keyName + clientID;
|
|
|
+ partName = metadataManager.getOzoneKey(volumeName, bucketName, keyName)
|
|
|
+ + clientID;
|
|
|
if (multipartKeyInfo == null) {
|
|
|
+ // This can occur when user started uploading part by the time commit
|
|
|
+ // of that part happens, in between the user might have requested
|
|
|
+ // abort multipart upload. If we just throw exception, then the data
|
|
|
+ // will not be garbage collected, so move this part to delete table
|
|
|
+ // and throw error
|
|
|
+ // 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);
|
|
|
} else {
|
|
@@ -886,4 +896,69 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void abortMultipartUpload(OmKeyArgs omKeyArgs) throws IOException {
|
|
|
+
|
|
|
+ Preconditions.checkNotNull(omKeyArgs);
|
|
|
+ String volumeName = omKeyArgs.getVolumeName();
|
|
|
+ String bucketName = omKeyArgs.getBucketName();
|
|
|
+ String keyName = omKeyArgs.getKeyName();
|
|
|
+ String uploadID = omKeyArgs.getMultipartUploadID();
|
|
|
+ Preconditions.checkNotNull(uploadID, "uploadID cannot be null");
|
|
|
+ metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
+
|
|
|
+ try {
|
|
|
+ String multipartKey = metadataManager.getMultipartKey(volumeName,
|
|
|
+ bucketName, keyName, uploadID);
|
|
|
+ OmMultipartKeyInfo multipartKeyInfo = metadataManager
|
|
|
+ .getMultipartInfoTable().get(multipartKey);
|
|
|
+ OmKeyInfo openKeyInfo = metadataManager.getOpenKeyTable().get(
|
|
|
+ multipartKey);
|
|
|
+
|
|
|
+ // If there is no entry in openKeyTable, then there is no multipart
|
|
|
+ // upload initiated for this key.
|
|
|
+ if (openKeyInfo == null) {
|
|
|
+ LOG.error("Abort Multipart Upload Failed: volume: " + volumeName +
|
|
|
+ "bucket: " + bucketName + "key: " + keyName + "with error no " +
|
|
|
+ "such uploadID:" + uploadID);
|
|
|
+ throw new OMException("Abort Multipart Upload Failed: volume: " +
|
|
|
+ volumeName + "bucket: " + bucketName + "key: " + keyName,
|
|
|
+ ResultCodes.NO_SUCH_MULTIPART_UPLOAD);
|
|
|
+ } else {
|
|
|
+ // Move all the parts to delete table
|
|
|
+ TreeMap<Integer, PartKeyInfo> partKeyInfoMap = multipartKeyInfo
|
|
|
+ .getPartKeyInfoList();
|
|
|
+ DBStore store = metadataManager.getStore();
|
|
|
+ try (BatchOperation batch = store.initBatchOperation()) {
|
|
|
+ for (Map.Entry<Integer, PartKeyInfo> partKeyInfoEntry : partKeyInfoMap
|
|
|
+ .entrySet()) {
|
|
|
+ PartKeyInfo partKeyInfo = partKeyInfoEntry.getValue();
|
|
|
+ OmKeyInfo currentKeyPartInfo = OmKeyInfo.getFromProtobuf(
|
|
|
+ partKeyInfo.getPartKeyInfo());
|
|
|
+ metadataManager.getDeletedTable().putWithBatch(batch,
|
|
|
+ partKeyInfo.getPartName(), currentKeyPartInfo);
|
|
|
+ }
|
|
|
+ // Finally delete the entry from the multipart info table and open
|
|
|
+ // key table
|
|
|
+ metadataManager.getMultipartInfoTable().deleteWithBatch(batch,
|
|
|
+ multipartKey);
|
|
|
+ metadataManager.getOpenKeyTable().deleteWithBatch(batch,
|
|
|
+ multipartKey);
|
|
|
+ store.commitBatchOperation(batch);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } catch (OMException ex) {
|
|
|
+ throw ex;
|
|
|
+ } catch (IOException ex) {
|
|
|
+ LOG.error("Abort Multipart Upload Failed: volume: " + volumeName +
|
|
|
+ "bucket: " + bucketName + "key: " + keyName, ex);
|
|
|
+ throw new OMException(ex.getMessage(), ResultCodes
|
|
|
+ .COMPLETE_MULTIPART_UPLOAD_FAILED);
|
|
|
+ } finally {
|
|
|
+ metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
|
|
+ }
|
|
|
+
|
|
|
+ }
|
|
|
+
|
|
|
}
|