|
@@ -65,7 +65,12 @@ import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadList;
|
|
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
|
|
import org.apache.hadoop.ozone.om.helpers.OmMultipartUploadListParts;
|
|
import org.apache.hadoop.ozone.om.helpers.OmPartInfo;
|
|
import org.apache.hadoop.ozone.om.helpers.OmPartInfo;
|
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
|
import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
|
|
-import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
|
|
|
|
|
|
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
|
+ .KeyArgs;
|
|
|
|
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
|
+ .KeyInfo;
|
|
|
|
+import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos
|
|
|
|
+ .KeyLocation;
|
|
import org.apache.hadoop.ozone.security.OzoneBlockTokenSecretManager;
|
|
import org.apache.hadoop.ozone.security.OzoneBlockTokenSecretManager;
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
@@ -233,7 +238,7 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public OmKeyLocationInfo addAllocatedBlock(OmKeyArgs args, long clientID,
|
|
public OmKeyLocationInfo addAllocatedBlock(OmKeyArgs args, long clientID,
|
|
- OzoneManagerProtocolProtos.KeyLocation keyLocation) throws IOException {
|
|
|
|
|
|
+ KeyLocation keyLocation) throws IOException {
|
|
Preconditions.checkNotNull(args);
|
|
Preconditions.checkNotNull(args);
|
|
Preconditions.checkNotNull(keyLocation);
|
|
Preconditions.checkNotNull(keyLocation);
|
|
|
|
|
|
@@ -518,10 +523,49 @@ public class KeyManagerImpl implements KeyManager {
|
|
allocateBlock(keyInfo, new ExcludeList(), args.getDataSize());
|
|
allocateBlock(keyInfo, new ExcludeList(), args.getDataSize());
|
|
keyInfo.appendNewBlocks(locationInfos);
|
|
keyInfo.appendNewBlocks(locationInfos);
|
|
}
|
|
}
|
|
- metadataManager.getOpenKeyTable().put(openKey, keyInfo);
|
|
|
|
|
|
+
|
|
|
|
+ // When OM is not managed via ratis we should write in to Om db in
|
|
|
|
+ // openKey call.
|
|
|
|
+ if (!isRatisEnabled) {
|
|
|
|
+ metadataManager.getOpenKeyTable().put(openKey, keyInfo);
|
|
|
|
+ }
|
|
return new OpenKeySession(currentTime, keyInfo, openVersion);
|
|
return new OpenKeySession(currentTime, keyInfo, openVersion);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public void applyOpenKey(KeyArgs omKeyArgs,
|
|
|
|
+ KeyInfo keyInfo, long clientID) throws IOException {
|
|
|
|
+ Preconditions.checkNotNull(omKeyArgs);
|
|
|
|
+ String volumeName = omKeyArgs.getVolumeName();
|
|
|
|
+ String bucketName = omKeyArgs.getBucketName();
|
|
|
|
+
|
|
|
|
+ // Do we need to call again validateBucket, as this is just called after
|
|
|
|
+ // start Transaction from applyTransaction. Can we remove this double
|
|
|
|
+ // check?
|
|
|
|
+ validateBucket(volumeName, bucketName);
|
|
|
|
+
|
|
|
|
+ metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
|
+ String keyName = omKeyArgs.getKeyName();
|
|
|
|
+
|
|
|
|
+ // TODO: here if on OM machines clocks are skewed and there is a chance
|
|
|
|
+ // for override of the openKey entries.
|
|
|
|
+ try {
|
|
|
|
+ String openKey = metadataManager.getOpenKey(
|
|
|
|
+ volumeName, bucketName, keyName, clientID);
|
|
|
|
+
|
|
|
|
+ OmKeyInfo omKeyInfo = OmKeyInfo.getFromProtobuf(keyInfo);
|
|
|
|
+
|
|
|
|
+ metadataManager.getOpenKeyTable().put(openKey,
|
|
|
|
+ omKeyInfo);
|
|
|
|
+ } catch (IOException ex) {
|
|
|
|
+ LOG.error("Apply Open Key failed for volume:{} bucket:{} key:{}",
|
|
|
|
+ volumeName, bucketName, keyName, ex);
|
|
|
|
+ throw new OMException(ex.getMessage(),
|
|
|
|
+ ResultCodes.KEY_ALLOCATION_ERROR);
|
|
|
|
+ } finally {
|
|
|
|
+ metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Create OmKeyInfo object.
|
|
* Create OmKeyInfo object.
|
|
* @param keyArgs
|
|
* @param keyArgs
|
|
@@ -826,17 +870,22 @@ public class KeyManagerImpl implements KeyManager {
|
|
@Override
|
|
@Override
|
|
public OmMultipartInfo initiateMultipartUpload(OmKeyArgs omKeyArgs) throws
|
|
public OmMultipartInfo initiateMultipartUpload(OmKeyArgs omKeyArgs) throws
|
|
IOException {
|
|
IOException {
|
|
- Preconditions.checkNotNull(omKeyArgs);
|
|
|
|
- String volumeName = omKeyArgs.getVolumeName();
|
|
|
|
- String bucketName = omKeyArgs.getBucketName();
|
|
|
|
- String keyName = omKeyArgs.getKeyName();
|
|
|
|
|
|
+ long time = Time.monotonicNowNanos();
|
|
|
|
+ String uploadID = UUID.randomUUID().toString() + "-" + time;
|
|
|
|
+ return applyInitiateMultipartUpload(omKeyArgs, uploadID);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public OmMultipartInfo applyInitiateMultipartUpload(OmKeyArgs keyArgs,
|
|
|
|
+ String multipartUploadID) throws IOException {
|
|
|
|
+ Preconditions.checkNotNull(keyArgs);
|
|
|
|
+ Preconditions.checkNotNull(multipartUploadID);
|
|
|
|
+ String volumeName = keyArgs.getVolumeName();
|
|
|
|
+ String bucketName = keyArgs.getBucketName();
|
|
|
|
+ String keyName = keyArgs.getKeyName();
|
|
|
|
|
|
metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
validateS3Bucket(volumeName, bucketName);
|
|
validateS3Bucket(volumeName, bucketName);
|
|
try {
|
|
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
|
|
// 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
|
|
// perform multipart upload on the same key, each will try to upload, who
|
|
@@ -852,24 +901,24 @@ public class KeyManagerImpl implements KeyManager {
|
|
// new uploadId is returned.
|
|
// new uploadId is returned.
|
|
|
|
|
|
String multipartKey = metadataManager.getMultipartKey(volumeName,
|
|
String multipartKey = metadataManager.getMultipartKey(volumeName,
|
|
- bucketName, keyName, uploadID);
|
|
|
|
|
|
+ bucketName, keyName, multipartUploadID);
|
|
|
|
|
|
// Not checking if there is an already key for this in the keyTable, as
|
|
// Not checking if there is an already key for this in the keyTable, as
|
|
// during final complete multipart upload we take care of this.
|
|
// during final complete multipart upload we take care of this.
|
|
|
|
|
|
|
|
|
|
Map<Integer, PartKeyInfo> partKeyInfoMap = new HashMap<>();
|
|
Map<Integer, PartKeyInfo> partKeyInfoMap = new HashMap<>();
|
|
- OmMultipartKeyInfo multipartKeyInfo = new OmMultipartKeyInfo(uploadID,
|
|
|
|
- partKeyInfoMap);
|
|
|
|
|
|
+ OmMultipartKeyInfo multipartKeyInfo = new OmMultipartKeyInfo(
|
|
|
|
+ multipartUploadID, partKeyInfoMap);
|
|
List<OmKeyLocationInfo> locations = new ArrayList<>();
|
|
List<OmKeyLocationInfo> locations = new ArrayList<>();
|
|
OmKeyInfo omKeyInfo = new OmKeyInfo.Builder()
|
|
OmKeyInfo omKeyInfo = new OmKeyInfo.Builder()
|
|
- .setVolumeName(omKeyArgs.getVolumeName())
|
|
|
|
- .setBucketName(omKeyArgs.getBucketName())
|
|
|
|
- .setKeyName(omKeyArgs.getKeyName())
|
|
|
|
|
|
+ .setVolumeName(keyArgs.getVolumeName())
|
|
|
|
+ .setBucketName(keyArgs.getBucketName())
|
|
|
|
+ .setKeyName(keyArgs.getKeyName())
|
|
.setCreationTime(Time.now())
|
|
.setCreationTime(Time.now())
|
|
.setModificationTime(Time.now())
|
|
.setModificationTime(Time.now())
|
|
- .setReplicationType(omKeyArgs.getType())
|
|
|
|
- .setReplicationFactor(omKeyArgs.getFactor())
|
|
|
|
|
|
+ .setReplicationType(keyArgs.getType())
|
|
|
|
+ .setReplicationFactor(keyArgs.getFactor())
|
|
.setOmKeyLocationInfos(Collections.singletonList(
|
|
.setOmKeyLocationInfos(Collections.singletonList(
|
|
new OmKeyLocationInfoGroup(0, locations)))
|
|
new OmKeyLocationInfoGroup(0, locations)))
|
|
.build();
|
|
.build();
|
|
@@ -882,11 +931,12 @@ public class KeyManagerImpl implements KeyManager {
|
|
metadataManager.getOpenKeyTable().putWithBatch(batch,
|
|
metadataManager.getOpenKeyTable().putWithBatch(batch,
|
|
multipartKey, omKeyInfo);
|
|
multipartKey, omKeyInfo);
|
|
store.commitBatchOperation(batch);
|
|
store.commitBatchOperation(batch);
|
|
- return new OmMultipartInfo(volumeName, bucketName, keyName, uploadID);
|
|
|
|
|
|
+ return new OmMultipartInfo(volumeName, bucketName, keyName,
|
|
|
|
+ multipartUploadID);
|
|
}
|
|
}
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
LOG.error("Initiate Multipart upload Failed for volume:{} bucket:{} " +
|
|
LOG.error("Initiate Multipart upload Failed for volume:{} bucket:{} " +
|
|
- "key:{}", volumeName, bucketName, keyName, ex);
|
|
|
|
|
|
+ "key:{}", volumeName, bucketName, keyName, ex);
|
|
throw new OMException(ex.getMessage(),
|
|
throw new OMException(ex.getMessage(),
|
|
ResultCodes.INITIATE_MULTIPART_UPLOAD_ERROR);
|
|
ResultCodes.INITIATE_MULTIPART_UPLOAD_ERROR);
|
|
} finally {
|
|
} finally {
|