|
@@ -20,6 +20,9 @@ import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.HashMap;
|
|
|
+import java.util.UUID;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
@@ -37,7 +40,11 @@ import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmMultipartInfo;
|
|
|
+import org.apache.hadoop.ozone.om.helpers.OmMultipartKeyInfo;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
|
|
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
+ .PartKeyInfo;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
import org.apache.hadoop.utils.BackgroundService;
|
|
|
import org.apache.hadoop.utils.db.BatchOperation;
|
|
@@ -54,6 +61,7 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MA
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
|
|
|
+
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
@@ -532,4 +540,74 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
public BackgroundService getDeletingService() {
|
|
|
return keyDeletingService;
|
|
|
}
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public OmMultipartInfo initiateMultipartUpload(OmKeyArgs omKeyArgs) throws
|
|
|
+ IOException {
|
|
|
+ Preconditions.checkNotNull(omKeyArgs);
|
|
|
+ String volumeName = omKeyArgs.getVolumeName();
|
|
|
+ String bucketName = omKeyArgs.getBucketName();
|
|
|
+ String keyName = omKeyArgs.getKeyName();
|
|
|
+
|
|
|
+ metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
+ try {
|
|
|
+ long time = Time.monotonicNowNanos();
|
|
|
+ String uploadID = UUID.randomUUID().toString() + "-" + Long.toString(
|
|
|
+ time);
|
|
|
+
|
|
|
+ // We are adding uploadId to key, because if multiple users try to
|
|
|
+ // perform multipart upload on the same key, each will try to upload, who
|
|
|
+ // ever finally commit the key, we see that key in ozone. Suppose if we
|
|
|
+ // don't add id, and use the same key /volume/bucket/key, when multiple
|
|
|
+ // users try to upload the key, we update the parts of the key's from
|
|
|
+ // multiple users to same key, and the key output can be a mix of the
|
|
|
+ // parts from multiple users.
|
|
|
+
|
|
|
+ // So on same key if multiple time multipart upload is initiated we
|
|
|
+ // store multiple entries in the openKey Table.
|
|
|
+ // Checked AWS S3, when we try to run multipart upload, each time a
|
|
|
+ // new uploadId is returned.
|
|
|
+
|
|
|
+ String multipartKey = metadataManager.getMultipartKey(volumeName,
|
|
|
+ bucketName, keyName, uploadID);
|
|
|
+
|
|
|
+ // Not checking if there is an already key for this in the keyTable, as
|
|
|
+ // during final complete multipart upload we take care of this.
|
|
|
+
|
|
|
+
|
|
|
+ Map<Integer, PartKeyInfo> partKeyInfoMap = new HashMap<>();
|
|
|
+ OmMultipartKeyInfo multipartKeyInfo = new OmMultipartKeyInfo(uploadID,
|
|
|
+ partKeyInfoMap);
|
|
|
+ List<OmKeyLocationInfo> locations = new ArrayList<>();
|
|
|
+ OmKeyInfo omKeyInfo = new OmKeyInfo.Builder()
|
|
|
+ .setVolumeName(omKeyArgs.getVolumeName())
|
|
|
+ .setBucketName(omKeyArgs.getBucketName())
|
|
|
+ .setKeyName(omKeyArgs.getKeyName())
|
|
|
+ .setCreationTime(Time.now())
|
|
|
+ .setModificationTime(Time.now())
|
|
|
+ .setReplicationType(omKeyArgs.getType())
|
|
|
+ .setReplicationFactor(omKeyArgs.getFactor())
|
|
|
+ .setOmKeyLocationInfos(Collections.singletonList(
|
|
|
+ new OmKeyLocationInfoGroup(0, locations)))
|
|
|
+ .build();
|
|
|
+ DBStore store = metadataManager.getStore();
|
|
|
+ try (BatchOperation batch = store.initBatchOperation()) {
|
|
|
+ // Create an entry in open key table and multipart info table for
|
|
|
+ // this key.
|
|
|
+ metadataManager.getMultipartInfoTable().putWithBatch(batch,
|
|
|
+ multipartKey, multipartKeyInfo);
|
|
|
+ metadataManager.getOpenKeyTable().putWithBatch(batch,
|
|
|
+ multipartKey, omKeyInfo);
|
|
|
+ store.commitBatchOperation(batch);
|
|
|
+ return new OmMultipartInfo(volumeName, bucketName, keyName, uploadID);
|
|
|
+ }
|
|
|
+ } catch (IOException ex) {
|
|
|
+ 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);
|
|
|
+ } finally {
|
|
|
+ metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|