|
@@ -26,6 +26,7 @@ import org.apache.hadoop.ozone.ksm.helpers.KsmKeyLocationInfo;
|
|
|
import org.apache.hadoop.conf.OzoneConfiguration;
|
|
|
import org.apache.hadoop.ozone.ksm.exceptions.KSMException;
|
|
|
import org.apache.hadoop.ozone.ksm.exceptions.KSMException.ResultCodes;
|
|
|
+import org.apache.hadoop.ozone.ksm.helpers.OpenKeySession;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.KeySpaceManagerProtocolProtos.KeyInfo;
|
|
|
import org.apache.hadoop.scm.container.common.helpers.AllocatedBlock;
|
|
|
import org.apache.hadoop.scm.protocol.ScmBlockLocationProtocol;
|
|
@@ -39,6 +40,7 @@ import org.slf4j.LoggerFactory;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
|
+import java.util.Random;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.DFS_CONTAINER_RATIS_ENABLED_DEFAULT;
|
|
@@ -47,6 +49,8 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVI
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_MS_DEFAULT;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE;
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_DEFAULT;
|
|
|
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SCM_BLOCK_SIZE_IN_MB;
|
|
|
import org.apache.hadoop.ozone.protocol.proto.OzoneProtos.ReplicationType;
|
|
@@ -69,6 +73,9 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
private final boolean useRatis;
|
|
|
private final BackgroundService keyDeletingService;
|
|
|
|
|
|
+ private final long preallocateMax;
|
|
|
+ private final Random random;
|
|
|
+
|
|
|
public KeyManagerImpl(ScmBlockLocationProtocol scmBlockClient,
|
|
|
KSMMetadataManager metadataManager, OzoneConfiguration conf) {
|
|
|
this.scmBlockClient = scmBlockClient;
|
|
@@ -83,8 +90,12 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
long serviceTimeout = conf.getTimeDuration(
|
|
|
OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
|
|
|
OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS);
|
|
|
+ this.preallocateMax = conf.getLong(
|
|
|
+ OZONE_KEY_PREALLOCATION_MAXSIZE,
|
|
|
+ OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT);
|
|
|
keyDeletingService = new KeyDeletingService(
|
|
|
scmBlockClient, this, svcInterval, serviceTimeout, conf);
|
|
|
+ random = new Random();
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -97,8 +108,28 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
keyDeletingService.shutdown();
|
|
|
}
|
|
|
|
|
|
+ private void validateBucket(String volumeName, String bucketName)
|
|
|
+ throws IOException {
|
|
|
+ byte[] volumeKey = metadataManager.getVolumeKey(volumeName);
|
|
|
+ byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
|
|
|
+
|
|
|
+ //Check if the volume exists
|
|
|
+ if(metadataManager.get(volumeKey) == null) {
|
|
|
+ LOG.error("volume not found: {}", volumeName);
|
|
|
+ throw new KSMException("Volume not found",
|
|
|
+ KSMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
|
|
+ }
|
|
|
+ //Check if bucket already exists
|
|
|
+ if(metadataManager.get(bucketKey) == null) {
|
|
|
+ LOG.error("bucket not found: {}/{} ", volumeName, bucketName);
|
|
|
+ throw new KSMException("Bucket not found",
|
|
|
+ KSMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
- public KsmKeyInfo allocateKey(KsmKeyArgs args) throws IOException {
|
|
|
+ public KsmKeyLocationInfo allocateBlock(KsmKeyArgs args, int clientID)
|
|
|
+ throws IOException {
|
|
|
Preconditions.checkNotNull(args);
|
|
|
metadataManager.writeLock().lock();
|
|
|
String volumeName = args.getVolumeName();
|
|
@@ -118,80 +149,162 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
- byte[] volumeKey = metadataManager.getVolumeKey(volumeName);
|
|
|
- byte[] bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
|
|
|
- byte[] keyKey =
|
|
|
- metadataManager.getDBKeyBytes(volumeName, bucketName, keyName);
|
|
|
-
|
|
|
- //Check if the volume exists
|
|
|
- if (metadataManager.get(volumeKey) == null) {
|
|
|
- LOG.debug("volume not found: {}", volumeName);
|
|
|
- throw new KSMException("Volume not found",
|
|
|
- KSMException.ResultCodes.FAILED_VOLUME_NOT_FOUND);
|
|
|
- }
|
|
|
- //Check if bucket already exists
|
|
|
- if (metadataManager.get(bucketKey) == null) {
|
|
|
- LOG.debug("bucket not found: {}/{} ", volumeName, bucketName);
|
|
|
- throw new KSMException("Bucket not found",
|
|
|
- KSMException.ResultCodes.FAILED_BUCKET_NOT_FOUND);
|
|
|
+ validateBucket(volumeName, bucketName);
|
|
|
+ String objectKey = metadataManager.getKeyWithDBPrefix(
|
|
|
+ volumeName, bucketName, keyName);
|
|
|
+ byte[] openKey = metadataManager.getOpenKeyNameBytes(objectKey, clientID);
|
|
|
+ byte[] keyData = metadataManager.get(openKey);
|
|
|
+ if (keyData == null) {
|
|
|
+ LOG.error("Allocate block for a key not in open status in meta store " +
|
|
|
+ objectKey + " with ID " + clientID);
|
|
|
+ throw new KSMException("Open Key not found",
|
|
|
+ KSMException.ResultCodes.FAILED_KEY_NOT_FOUND);
|
|
|
}
|
|
|
+ AllocatedBlock allocatedBlock =
|
|
|
+ scmBlockClient.allocateBlock(scmBlockSize, type, factor);
|
|
|
+ KsmKeyInfo keyInfo =
|
|
|
+ KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(keyData));
|
|
|
+ KsmKeyLocationInfo info = new KsmKeyLocationInfo.Builder()
|
|
|
+ .setContainerName(allocatedBlock.getPipeline().getContainerName())
|
|
|
+ .setBlockID(allocatedBlock.getKey())
|
|
|
+ .setShouldCreateContainer(allocatedBlock.getCreateContainer())
|
|
|
+ .setLength(scmBlockSize)
|
|
|
+ .setOffset(0)
|
|
|
+ .setIndex(keyInfo.getKeyLocationList().size())
|
|
|
+ .build();
|
|
|
+ keyInfo.appendKeyLocation(info);
|
|
|
+ metadataManager.put(openKey, keyInfo.getProtobuf().toByteArray());
|
|
|
+ return info;
|
|
|
+ } finally {
|
|
|
+ metadataManager.writeLock().unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
|
|
|
- // TODO: Garbage collect deleted blocks due to overwrite of a key.
|
|
|
- // FIXME: BUG: Please see HDFS-11922.
|
|
|
- // If user overwrites a key, then we are letting it pass without
|
|
|
- // corresponding process.
|
|
|
- // In reality we need to garbage collect those blocks by telling SCM to
|
|
|
- // clean up those blocks when it can. Right now making this change
|
|
|
- // allows us to pass tests that expect ozone can overwrite a key.
|
|
|
+ @Override
|
|
|
+ public OpenKeySession openKey(KsmKeyArgs args) throws IOException {
|
|
|
+ Preconditions.checkNotNull(args);
|
|
|
+ metadataManager.writeLock().lock();
|
|
|
+ String volumeName = args.getVolumeName();
|
|
|
+ String bucketName = args.getBucketName();
|
|
|
+ String keyName = args.getKeyName();
|
|
|
+ ReplicationFactor factor = args.getFactor();
|
|
|
+ ReplicationType type = args.getType();
|
|
|
|
|
|
- // When we talk to SCM make sure that we ask for at least a byte in the
|
|
|
- // block. This way even if the call is for a zero length key, we back it
|
|
|
- // with a actual SCM block.
|
|
|
- // TODO : Review this decision later. We can get away with only a
|
|
|
- // metadata entry in case of 0 length key.
|
|
|
- long targetSize = args.getDataSize();
|
|
|
- List<KsmKeyLocationInfo> subKeyInfos = new ArrayList<>();
|
|
|
- int idx = 0;
|
|
|
- long offset = 0;
|
|
|
+ // If user does not specify a replication strategy or
|
|
|
+ // replication factor, KSM will use defaults.
|
|
|
+ if(factor == null) {
|
|
|
+ factor = useRatis ? ReplicationFactor.THREE: ReplicationFactor.ONE;
|
|
|
+ }
|
|
|
|
|
|
- // in case targetSize == 0, subKeyInfos will be an empty list
|
|
|
- while (targetSize > 0) {
|
|
|
- long allocateSize = Math.min(targetSize, scmBlockSize);
|
|
|
+ if(type == null) {
|
|
|
+ type = useRatis ? ReplicationType.RATIS : ReplicationType.STAND_ALONE;
|
|
|
+ }
|
|
|
+
|
|
|
+ try {
|
|
|
+ validateBucket(volumeName, bucketName);
|
|
|
+ long requestedSize = Math.min(preallocateMax, args.getDataSize());
|
|
|
+ List<KsmKeyLocationInfo> locations = new ArrayList<>();
|
|
|
+ String objectKey = metadataManager.getKeyWithDBPrefix(
|
|
|
+ volumeName, bucketName, keyName);
|
|
|
+ // requested size is not required but more like a optimization:
|
|
|
+ // SCM looks at the requested, if it 0, no block will be allocated at
|
|
|
+ // the point, if client needs more blocks, client can always call
|
|
|
+ // allocateBlock. But if requested size is not 0, KSM will preallocate
|
|
|
+ // some blocks and piggyback to client, to save RPC calls.
|
|
|
+ int idx = 0;
|
|
|
+ while (requestedSize > 0) {
|
|
|
+ long allocateSize = Math.min(scmBlockSize, requestedSize);
|
|
|
AllocatedBlock allocatedBlock =
|
|
|
scmBlockClient.allocateBlock(allocateSize, type, factor);
|
|
|
KsmKeyLocationInfo subKeyInfo = new KsmKeyLocationInfo.Builder()
|
|
|
.setContainerName(allocatedBlock.getPipeline().getContainerName())
|
|
|
.setBlockID(allocatedBlock.getKey())
|
|
|
.setShouldCreateContainer(allocatedBlock.getCreateContainer())
|
|
|
- .setIndex(idx)
|
|
|
+ .setIndex(idx++)
|
|
|
.setLength(allocateSize)
|
|
|
- .setOffset(offset)
|
|
|
+ .setOffset(0)
|
|
|
.build();
|
|
|
- idx += 1;
|
|
|
- offset += allocateSize;
|
|
|
- targetSize -= allocateSize;
|
|
|
- subKeyInfos.add(subKeyInfo);
|
|
|
+ locations.add(subKeyInfo);
|
|
|
+ requestedSize -= allocateSize;
|
|
|
}
|
|
|
-
|
|
|
long currentTime = Time.now();
|
|
|
- KsmKeyInfo keyBlock = new KsmKeyInfo.Builder()
|
|
|
+ // 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;
|
|
|
+ KsmKeyInfo keyInfo = new KsmKeyInfo.Builder()
|
|
|
.setVolumeName(args.getVolumeName())
|
|
|
.setBucketName(args.getBucketName())
|
|
|
.setKeyName(args.getKeyName())
|
|
|
- .setDataSize(args.getDataSize())
|
|
|
- .setKsmKeyLocationInfos(subKeyInfos)
|
|
|
+ .setKsmKeyLocationInfos(locations)
|
|
|
.setCreationTime(currentTime)
|
|
|
.setModificationTime(currentTime)
|
|
|
+ .setDataSize(size)
|
|
|
.build();
|
|
|
- metadataManager.put(keyKey, keyBlock.getProtobuf().toByteArray());
|
|
|
- LOG.debug("Key {} allocated in volume {} bucket {}", keyName, volumeName,
|
|
|
- bucketName);
|
|
|
- return keyBlock;
|
|
|
+ // Generate a random ID which is not already in meta db.
|
|
|
+ int id = -1;
|
|
|
+ // in general this should finish in a couple times at most. putting some
|
|
|
+ // arbitrary large number here to avoid dead loop.
|
|
|
+ for (int j = 0; j < 10000; j++) {
|
|
|
+ id = random.nextInt();
|
|
|
+ byte[] openKey = metadataManager.getOpenKeyNameBytes(objectKey, id);
|
|
|
+ if (metadataManager.get(openKey) == null) {
|
|
|
+ metadataManager.put(openKey, keyInfo.getProtobuf().toByteArray());
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (id == -1) {
|
|
|
+ throw new IOException("Failed to find a usable id for " + objectKey);
|
|
|
+ }
|
|
|
+ LOG.debug("Key {} allocated in volume {} bucket {}",
|
|
|
+ keyName, volumeName, bucketName);
|
|
|
+ return new OpenKeySession(id, keyInfo);
|
|
|
+ } catch (KSMException e) {
|
|
|
+ throw e;
|
|
|
+ } catch (IOException ex) {
|
|
|
+ if (!(ex instanceof KSMException)) {
|
|
|
+ LOG.error("Key open failed for volume:{} bucket:{} key:{}",
|
|
|
+ volumeName, bucketName, keyName, ex);
|
|
|
+ }
|
|
|
+ throw new KSMException(ex.getMessage(),
|
|
|
+ KSMException.ResultCodes.FAILED_KEY_ALLOCATION);
|
|
|
+ } finally {
|
|
|
+ metadataManager.writeLock().unlock();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void commitKey(KsmKeyArgs args, int clientID) throws IOException {
|
|
|
+ Preconditions.checkNotNull(args);
|
|
|
+ metadataManager.writeLock().lock();
|
|
|
+ String volumeName = args.getVolumeName();
|
|
|
+ String bucketName = args.getBucketName();
|
|
|
+ String keyName = args.getKeyName();
|
|
|
+ try {
|
|
|
+ validateBucket(volumeName, bucketName);
|
|
|
+ String objectKey = metadataManager.getKeyWithDBPrefix(
|
|
|
+ volumeName, bucketName, keyName);
|
|
|
+ byte[] objectKeyBytes = metadataManager.getDBKeyBytes(volumeName,
|
|
|
+ bucketName, keyName);
|
|
|
+ byte[] openKey = metadataManager.getOpenKeyNameBytes(objectKey, clientID);
|
|
|
+ byte[] openKeyData = metadataManager.get(openKey);
|
|
|
+ if (openKeyData == null) {
|
|
|
+ throw new KSMException("Commit a key without corresponding entry " +
|
|
|
+ DFSUtil.bytes2String(openKey), ResultCodes.FAILED_KEY_NOT_FOUND);
|
|
|
+ }
|
|
|
+ KsmKeyInfo keyInfo =
|
|
|
+ KsmKeyInfo.getFromProtobuf(KeyInfo.parseFrom(openKeyData));
|
|
|
+ keyInfo.setDataSize(args.getDataSize());
|
|
|
+ BatchOperation batch = new BatchOperation();
|
|
|
+ batch.delete(openKey);
|
|
|
+ batch.put(objectKeyBytes, keyInfo.getProtobuf().toByteArray());
|
|
|
+ metadataManager.writeBatch(batch);
|
|
|
} catch (KSMException e) {
|
|
|
throw e;
|
|
|
} catch (IOException ex) {
|
|
|
if (!(ex instanceof KSMException)) {
|
|
|
- LOG.error("Key allocation failed for volume:{} bucket:{} key:{}",
|
|
|
+ LOG.error("Key commit failed for volume:{} bucket:{} key:{}",
|
|
|
volumeName, bucketName, keyName, ex);
|
|
|
}
|
|
|
throw new KSMException(ex.getMessage(),
|