|
@@ -84,6 +84,7 @@ 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.UniqueId;
|
|
|
import org.apache.hadoop.utils.db.BatchOperation;
|
|
|
import org.apache.hadoop.utils.db.DBStore;
|
|
|
import org.apache.hadoop.utils.db.CodecRegistry;
|
|
@@ -406,10 +407,29 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
String keyName = args.getKeyName();
|
|
|
validateBucket(volumeName, bucketName);
|
|
|
|
|
|
- long currentTime = Time.monotonicNowNanos();
|
|
|
+ long currentTime = UniqueId.next();
|
|
|
OmKeyInfo keyInfo;
|
|
|
- String openKey;
|
|
|
long openVersion;
|
|
|
+ // NOTE size of a key is not a hard limit on anything, it is a value that
|
|
|
+ // client should expect, in terms of current size of key. If client sets
|
|
|
+ // a value, then this value is used, otherwise, we allocate a single
|
|
|
+ // block which is the current size, if read by the client.
|
|
|
+ final long size = args.getDataSize() >= 0 ?
|
|
|
+ args.getDataSize() : scmBlockSize;
|
|
|
+ final List<OmKeyLocationInfo> locations = new ArrayList<>();
|
|
|
+
|
|
|
+ ReplicationFactor factor = args.getFactor();
|
|
|
+ if (factor == null) {
|
|
|
+ factor = useRatis ? ReplicationFactor.THREE : ReplicationFactor.ONE;
|
|
|
+ }
|
|
|
+
|
|
|
+ ReplicationType type = args.getType();
|
|
|
+ if (type == null) {
|
|
|
+ type = useRatis ? ReplicationType.RATIS : ReplicationType.STAND_ALONE;
|
|
|
+ }
|
|
|
+
|
|
|
+ String dbKeyName = metadataManager.getOzoneKey(
|
|
|
+ args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
|
|
FileEncryptionInfo encInfo;
|
|
|
|
|
@@ -417,37 +437,7 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
OmBucketInfo bucketInfo = getBucketInfo(volumeName, bucketName);
|
|
|
encInfo = getFileEncryptionInfo(bucketInfo);
|
|
|
- // NOTE size of a key is not a hard limit on anything, it is a value that
|
|
|
- // client should expect, in terms of current size of key. If client sets
|
|
|
- // a value, then this value is used, otherwise, we allocate a single
|
|
|
- // block which is the current size, if read by the client.
|
|
|
- long size = args.getDataSize() >= 0 ? args.getDataSize() : scmBlockSize;
|
|
|
- List<OmKeyLocationInfo> locations = new ArrayList<>();
|
|
|
- if (args.getIsMultipartKey()) {
|
|
|
- keyInfo = prepareMultipartKeyInfo(args, size, locations, encInfo);
|
|
|
- //TODO args.getMetadata
|
|
|
- } else {
|
|
|
- keyInfo = prepareKeyInfo(args, size, locations, encInfo);
|
|
|
- }
|
|
|
-
|
|
|
- openVersion = keyInfo.getLatestVersionLocations().getVersion();
|
|
|
- openKey = metadataManager.getOpenKey(
|
|
|
- volumeName, bucketName, keyName, currentTime);
|
|
|
- if (metadataManager.getOpenKeyTable().get(openKey) != null) {
|
|
|
- // This should not happen. If this condition is satisfied, it means
|
|
|
- // that we have generated a same openKeyId (i.e. currentTime) for two
|
|
|
- // different client who are trying to write the same key at the same
|
|
|
- // time. The chance of this happening is very, very minimal.
|
|
|
-
|
|
|
- // Do we really need this check? Can we avoid this to gain some
|
|
|
- // minor performance improvement?
|
|
|
- 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.", ResultCodes.KEY_ALLOCATION_ERROR);
|
|
|
- }
|
|
|
- LOG.debug("Key {} allocated in volume {} bucket {}",
|
|
|
- keyName, volumeName, bucketName);
|
|
|
+ keyInfo = prepareKeyInfo(args, dbKeyName, size, locations, encInfo);
|
|
|
} catch (OMException e) {
|
|
|
throw e;
|
|
|
} catch (IOException ex) {
|
|
@@ -457,7 +447,14 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
} finally {
|
|
|
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
|
|
}
|
|
|
-
|
|
|
+ if (keyInfo == null) {
|
|
|
+ // the key does not exist, create a new object, the new blocks are the
|
|
|
+ // version 0
|
|
|
+ keyInfo = createKeyInfo(args, locations, factor, type, size, encInfo);
|
|
|
+ }
|
|
|
+ openVersion = keyInfo.getLatestVersionLocations().getVersion();
|
|
|
+ LOG.debug("Key {} allocated in volume {} bucket {}",
|
|
|
+ keyName, volumeName, bucketName);
|
|
|
allocateBlockInKey(keyInfo, args.getDataSize(), currentTime);
|
|
|
return new OpenKeySession(currentTime, keyInfo, openVersion);
|
|
|
}
|
|
@@ -485,33 +482,21 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private OmKeyInfo prepareKeyInfo(OmKeyArgs args, long size,
|
|
|
+ private OmKeyInfo prepareKeyInfo(
|
|
|
+ OmKeyArgs keyArgs, String dbKeyName, long size,
|
|
|
List<OmKeyLocationInfo> locations, FileEncryptionInfo encInfo)
|
|
|
throws IOException {
|
|
|
- ReplicationFactor factor = args.getFactor();
|
|
|
- ReplicationType type = args.getType();
|
|
|
- OmKeyInfo keyInfo;
|
|
|
- // If user does not specify a replication strategy or
|
|
|
- // replication factor, OM will use defaults.
|
|
|
- if (factor == null) {
|
|
|
- factor = useRatis ? ReplicationFactor.THREE : ReplicationFactor.ONE;
|
|
|
- }
|
|
|
- if (type == null) {
|
|
|
- type = useRatis ? ReplicationType.RATIS : ReplicationType.STAND_ALONE;
|
|
|
- }
|
|
|
- String objectKey = metadataManager.getOzoneKey(
|
|
|
- args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
- keyInfo = metadataManager.getKeyTable().get(objectKey);
|
|
|
- if (keyInfo != null) {
|
|
|
+ OmKeyInfo keyInfo = null;
|
|
|
+ if (keyArgs.getIsMultipartKey()) {
|
|
|
+ keyInfo = prepareMultipartKeyInfo(keyArgs, size, locations, encInfo);
|
|
|
+ //TODO args.getMetadata
|
|
|
+ } else if (metadataManager.getKeyTable().isExist(dbKeyName)) {
|
|
|
+ keyInfo = metadataManager.getKeyTable().get(dbKeyName);
|
|
|
// the key already exist, the new blocks will be added as new version
|
|
|
// when locations.size = 0, the new version will have identical blocks
|
|
|
// as its previous version
|
|
|
keyInfo.addNewVersion(locations);
|
|
|
keyInfo.setDataSize(size + keyInfo.getDataSize());
|
|
|
- } else {
|
|
|
- // the key does not exist, create a new object, the new blocks are the
|
|
|
- // version 0
|
|
|
- keyInfo = createKeyInfo(args, locations, factor, type, size, encInfo);
|
|
|
}
|
|
|
return keyInfo;
|
|
|
}
|
|
@@ -618,13 +603,15 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
String volumeName = args.getVolumeName();
|
|
|
String bucketName = args.getBucketName();
|
|
|
String keyName = args.getKeyName();
|
|
|
- metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
+ List<OmKeyLocationInfo> locationInfoList = args.getLocationInfoList();
|
|
|
+ String objectKey = metadataManager
|
|
|
+ .getOzoneKey(volumeName, bucketName, keyName);
|
|
|
+ String openKey = metadataManager
|
|
|
+ .getOpenKey(volumeName, bucketName, keyName, clientID);
|
|
|
+ Preconditions.checkNotNull(locationInfoList);
|
|
|
try {
|
|
|
+ metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
validateBucket(volumeName, bucketName);
|
|
|
- String openKey = metadataManager.getOpenKey(volumeName, bucketName,
|
|
|
- keyName, clientID);
|
|
|
- String objectKey = metadataManager.getOzoneKey(
|
|
|
- volumeName, bucketName, keyName);
|
|
|
OmKeyInfo keyInfo = metadataManager.getOpenKeyTable().get(openKey);
|
|
|
if (keyInfo == null) {
|
|
|
throw new OMException("Commit a key without corresponding entry " +
|
|
@@ -632,8 +619,6 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
}
|
|
|
keyInfo.setDataSize(args.getDataSize());
|
|
|
keyInfo.setModificationTime(Time.now());
|
|
|
- List<OmKeyLocationInfo> locationInfoList = args.getLocationInfoList();
|
|
|
- Preconditions.checkNotNull(locationInfoList);
|
|
|
|
|
|
//update the block length for each block
|
|
|
keyInfo.updateLocationInfoList(locationInfoList);
|