|
@@ -17,6 +17,8 @@
|
|
|
package org.apache.hadoop.ozone.om;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.nio.file.Path;
|
|
|
+import java.nio.file.Paths;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collections;
|
|
|
import java.util.EnumSet;
|
|
@@ -201,17 +203,20 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
|
|
|
private void validateBucket(String volumeName, String bucketName)
|
|
|
throws IOException {
|
|
|
- String volumeKey = metadataManager.getVolumeKey(volumeName);
|
|
|
String bucketKey = metadataManager.getBucketKey(volumeName, bucketName);
|
|
|
-
|
|
|
- //Check if the volume exists
|
|
|
- if (metadataManager.getVolumeTable().get(volumeKey) == null) {
|
|
|
- LOG.error("volume not found: {}", volumeName);
|
|
|
- throw new OMException("Volume not found",
|
|
|
- OMException.ResultCodes.VOLUME_NOT_FOUND);
|
|
|
- }
|
|
|
- //Check if bucket already exists
|
|
|
+ // Check if bucket exists
|
|
|
if (metadataManager.getBucketTable().get(bucketKey) == null) {
|
|
|
+ String volumeKey = metadataManager.getVolumeKey(volumeName);
|
|
|
+ // If the volume also does not exist, we should throw volume not found
|
|
|
+ // exception
|
|
|
+ if (metadataManager.getVolumeTable().get(volumeKey) == null) {
|
|
|
+ LOG.error("volume not found: {}", volumeName);
|
|
|
+ throw new OMException("Volume not found",
|
|
|
+ OMException.ResultCodes.VOLUME_NOT_FOUND);
|
|
|
+ }
|
|
|
+
|
|
|
+ // if the volume exists but bucket does not exist, throw bucket not found
|
|
|
+ // exception
|
|
|
LOG.error("bucket not found: {}/{} ", volumeName, bucketName);
|
|
|
throw new OMException("Bucket not found",
|
|
|
OMException.ResultCodes.BUCKET_NOT_FOUND);
|
|
@@ -388,103 +393,39 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
return edek;
|
|
|
}
|
|
|
|
|
|
- @SuppressWarnings("checkstyle:methodlength")
|
|
|
@Override
|
|
|
public OpenKeySession openKey(OmKeyArgs args) throws IOException {
|
|
|
Preconditions.checkNotNull(args);
|
|
|
String volumeName = args.getVolumeName();
|
|
|
String bucketName = args.getBucketName();
|
|
|
+ String keyName = args.getKeyName();
|
|
|
validateBucket(volumeName, bucketName);
|
|
|
|
|
|
- metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
- String keyName = args.getKeyName();
|
|
|
- ReplicationFactor factor = args.getFactor();
|
|
|
- ReplicationType type = args.getType();
|
|
|
long currentTime = Time.monotonicNowNanos();
|
|
|
OmKeyInfo keyInfo;
|
|
|
String openKey;
|
|
|
long openVersion;
|
|
|
|
|
|
- FileEncryptionInfo encInfo = null;
|
|
|
- OmBucketInfo bucketInfo = getBucketInfo(volumeName, bucketName);
|
|
|
- BucketEncryptionKeyInfo ezInfo = bucketInfo.getEncryptionKeyInfo();
|
|
|
- if (ezInfo != null) {
|
|
|
- if (getKMSProvider() == null) {
|
|
|
- throw new OMException("Invalid KMS provider, check configuration " +
|
|
|
- CommonConfigurationKeys.HADOOP_SECURITY_KEY_PROVIDER_PATH,
|
|
|
- OMException.ResultCodes.INVALID_KMS_PROVIDER);
|
|
|
- }
|
|
|
-
|
|
|
- final String ezKeyName = ezInfo.getKeyName();
|
|
|
- EncryptedKeyVersion edek = generateEDEK(ezKeyName);
|
|
|
- encInfo = new FileEncryptionInfo(ezInfo.getSuite(), ezInfo.getVersion(),
|
|
|
- edek.getEncryptedKeyVersion().getMaterial(),
|
|
|
- edek.getEncryptedKeyIv(),
|
|
|
- ezKeyName, edek.getEncryptionKeyVersionName());
|
|
|
- }
|
|
|
+ FileEncryptionInfo encInfo;
|
|
|
|
|
|
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
|
|
|
- // error no such multipart upload.
|
|
|
- String uploadID = args.getMultipartUploadID();
|
|
|
- Preconditions.checkNotNull(uploadID);
|
|
|
- String multipartKey = metadataManager.getMultipartKey(volumeName,
|
|
|
- bucketName, keyName, uploadID);
|
|
|
- OmKeyInfo partKeyInfo = metadataManager.getOpenKeyTable().get(
|
|
|
- multipartKey);
|
|
|
- if (partKeyInfo == null) {
|
|
|
- throw new OMException("No such Multipart upload is with specified " +
|
|
|
- "uploadId " + uploadID,
|
|
|
- ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
|
|
|
- } else {
|
|
|
- factor = partKeyInfo.getFactor();
|
|
|
- type = partKeyInfo.getType();
|
|
|
- }
|
|
|
- } else {
|
|
|
- // 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;
|
|
|
- }
|
|
|
- }
|
|
|
- List<OmKeyLocationInfo> locations = new ArrayList<>();
|
|
|
- String objectKey = metadataManager.getOzoneKey(
|
|
|
- volumeName, bucketName, keyName);
|
|
|
+ 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.
|
|
|
+ // 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()) {
|
|
|
- // For this upload part we don't need to check in KeyTable. As this
|
|
|
- // is not an actual key, it is a part of the key.
|
|
|
- keyInfo = createKeyInfo(args, locations, factor, type, size, encInfo);
|
|
|
+ keyInfo = prepareMultipartKeyInfo(args, size, locations, encInfo);
|
|
|
//TODO args.getMetadata
|
|
|
- openVersion = 0;
|
|
|
} else {
|
|
|
- keyInfo = metadataManager.getKeyTable().get(objectKey);
|
|
|
- if (keyInfo != null) {
|
|
|
- // 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
|
|
|
- openVersion = 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);
|
|
|
- openVersion = 0;
|
|
|
- }
|
|
|
+ keyInfo = prepareKeyInfo(args, size, locations, encInfo);
|
|
|
}
|
|
|
+
|
|
|
+ openVersion = keyInfo.getLatestVersionLocations().getVersion();
|
|
|
openKey = metadataManager.getOpenKey(
|
|
|
volumeName, bucketName, keyName, currentTime);
|
|
|
if (metadataManager.getOpenKeyTable().get(openKey) != null) {
|
|
@@ -507,20 +448,28 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
} catch (IOException ex) {
|
|
|
LOG.error("Key open failed for volume:{} bucket:{} key:{}",
|
|
|
volumeName, bucketName, keyName, ex);
|
|
|
- throw new OMException(ex.getMessage(),
|
|
|
- ResultCodes.KEY_ALLOCATION_ERROR);
|
|
|
+ throw new OMException(ex.getMessage(), ResultCodes.KEY_ALLOCATION_ERROR);
|
|
|
} finally {
|
|
|
metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
|
|
}
|
|
|
|
|
|
+ allocateBlockInKey(keyInfo, args.getDataSize(), currentTime);
|
|
|
+ return new OpenKeySession(currentTime, keyInfo, openVersion);
|
|
|
+ }
|
|
|
+
|
|
|
+ private void allocateBlockInKey(OmKeyInfo keyInfo, long size, long sessionId)
|
|
|
+ throws IOException {
|
|
|
+ String openKey = metadataManager
|
|
|
+ .getOpenKey(keyInfo.getVolumeName(), keyInfo.getBucketName(),
|
|
|
+ keyInfo.getKeyName(), sessionId);
|
|
|
// 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, OM will preallocate
|
|
|
// some blocks and piggyback to client, to save RPC calls.
|
|
|
- if (args.getDataSize() > 0) {
|
|
|
+ if (size > 0) {
|
|
|
List<OmKeyLocationInfo> locationInfos =
|
|
|
- allocateBlock(keyInfo, new ExcludeList(), args.getDataSize());
|
|
|
+ allocateBlock(keyInfo, new ExcludeList(), size);
|
|
|
keyInfo.appendNewBlocks(locationInfos);
|
|
|
}
|
|
|
|
|
@@ -529,7 +478,69 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
if (!isRatisEnabled) {
|
|
|
metadataManager.getOpenKeyTable().put(openKey, keyInfo);
|
|
|
}
|
|
|
- return new OpenKeySession(currentTime, keyInfo, openVersion);
|
|
|
+ }
|
|
|
+
|
|
|
+ private OmKeyInfo prepareKeyInfo(OmKeyArgs args, 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) {
|
|
|
+ // 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;
|
|
|
+ }
|
|
|
+
|
|
|
+ private OmKeyInfo prepareMultipartKeyInfo(OmKeyArgs args, long size,
|
|
|
+ List<OmKeyLocationInfo> locations, FileEncryptionInfo encInfo)
|
|
|
+ throws IOException {
|
|
|
+ ReplicationFactor factor;
|
|
|
+ ReplicationType type;
|
|
|
+
|
|
|
+ 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
|
|
|
+ // error no such multipart upload.
|
|
|
+ String uploadID = args.getMultipartUploadID();
|
|
|
+ Preconditions.checkNotNull(uploadID);
|
|
|
+ String multipartKey = metadataManager
|
|
|
+ .getMultipartKey(args.getVolumeName(), args.getBucketName(),
|
|
|
+ args.getKeyName(), uploadID);
|
|
|
+ OmKeyInfo partKeyInfo = metadataManager.getOpenKeyTable().get(
|
|
|
+ multipartKey);
|
|
|
+ if (partKeyInfo == null) {
|
|
|
+ throw new OMException("No such Multipart upload is with specified " +
|
|
|
+ "uploadId " + uploadID,
|
|
|
+ ResultCodes.NO_SUCH_MULTIPART_UPLOAD_ERROR);
|
|
|
+ } else {
|
|
|
+ factor = partKeyInfo.getFactor();
|
|
|
+ type = partKeyInfo.getType();
|
|
|
+ }
|
|
|
+ // For this upload part we don't need to check in KeyTable. As this
|
|
|
+ // is not an actual key, it is a part of the key.
|
|
|
+ return createKeyInfo(args, locations, factor, type, size, encInfo);
|
|
|
}
|
|
|
|
|
|
public void applyOpenKey(KeyArgs omKeyArgs,
|
|
@@ -1329,18 +1340,27 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public OzoneFileStatus getFileStatus(String volumeName, String bucketName,
|
|
|
- String keyName) throws IOException {
|
|
|
- Preconditions.checkNotNull(volumeName);
|
|
|
- Preconditions.checkNotNull(bucketName);
|
|
|
- Preconditions.checkNotNull(keyName);
|
|
|
+ /**
|
|
|
+ * OzoneFS api to get file status for an entry.
|
|
|
+ *
|
|
|
+ * @param args Key args
|
|
|
+ * @throws OMException if file does not exist
|
|
|
+ * if bucket does not exist
|
|
|
+ * @throws IOException if there is error in the db
|
|
|
+ * invalid arguments
|
|
|
+ */
|
|
|
+ public OzoneFileStatus getFileStatus(OmKeyArgs args) throws IOException {
|
|
|
+ Preconditions.checkNotNull(args, "Key args can not be null");
|
|
|
+ String volumeName = args.getVolumeName();
|
|
|
+ String bucketName = args.getBucketName();
|
|
|
+ String keyName = args.getKeyName();
|
|
|
|
|
|
metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
try {
|
|
|
// Check if this is the root of the filesystem.
|
|
|
if (keyName.length() == 0) {
|
|
|
validateBucket(volumeName, bucketName);
|
|
|
- return new OzoneFileStatus(3, scmBlockSize, keyName);
|
|
|
+ return new OzoneFileStatus(keyName);
|
|
|
}
|
|
|
|
|
|
//Check if the key is a file.
|
|
@@ -1363,7 +1383,7 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
List<OmKeyInfo> keys = metadataManager.listKeys(volumeName, bucketName,
|
|
|
null, dirKey, 1);
|
|
|
if (keys.iterator().hasNext()) {
|
|
|
- return new OzoneFileStatus(3, scmBlockSize, keyName);
|
|
|
+ return new OzoneFileStatus(keyName);
|
|
|
}
|
|
|
|
|
|
LOG.debug("Unable to get file status for the key: volume:" + volumeName +
|
|
@@ -1377,6 +1397,226 @@ public class KeyManagerImpl implements KeyManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Ozone FS api to create a directory. Parent directories if do not exist
|
|
|
+ * are created for the input directory.
|
|
|
+ *
|
|
|
+ * @param args Key args
|
|
|
+ * @throws OMException if any entry in the path exists as a file
|
|
|
+ * if bucket does not exist
|
|
|
+ * @throws IOException if there is error in the db
|
|
|
+ * invalid arguments
|
|
|
+ */
|
|
|
+ public void createDirectory(OmKeyArgs args) throws IOException {
|
|
|
+ Preconditions.checkNotNull(args, "Key args can not be null");
|
|
|
+ String volumeName = args.getVolumeName();
|
|
|
+ String bucketName = args.getBucketName();
|
|
|
+ String keyName = args.getKeyName();
|
|
|
+
|
|
|
+ try {
|
|
|
+ metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
+
|
|
|
+ // verify bucket exists
|
|
|
+ OmBucketInfo bucketInfo = getBucketInfo(volumeName, bucketName);
|
|
|
+
|
|
|
+ // Check if this is the root of the filesystem.
|
|
|
+ if (keyName.length() == 0) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ verifyNoFilesInPath(volumeName, bucketName, Paths.get(keyName), false);
|
|
|
+ String dir = addTrailingSlashIfNeeded(keyName);
|
|
|
+ String dirDbKey =
|
|
|
+ metadataManager.getOzoneKey(volumeName, bucketName, dir);
|
|
|
+ FileEncryptionInfo encInfo = getFileEncryptionInfo(bucketInfo);
|
|
|
+ OmKeyInfo dirDbKeyInfo =
|
|
|
+ createDirectoryKeyInfo(volumeName, bucketName, dir, new ArrayList<>(),
|
|
|
+ ReplicationFactor.ONE, ReplicationType.RATIS, encInfo);
|
|
|
+ metadataManager.getKeyTable().put(dirDbKey, dirDbKeyInfo);
|
|
|
+
|
|
|
+ } finally {
|
|
|
+ metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private OmKeyInfo createDirectoryKeyInfo(String volumeName, String bucketName,
|
|
|
+ String keyName, List<OmKeyLocationInfo> locations,
|
|
|
+ ReplicationFactor factor, ReplicationType type,
|
|
|
+ FileEncryptionInfo encInfo) {
|
|
|
+ return new OmKeyInfo.Builder()
|
|
|
+ .setVolumeName(volumeName)
|
|
|
+ .setBucketName(bucketName)
|
|
|
+ .setKeyName(keyName)
|
|
|
+ .setOmKeyLocationInfos(Collections.singletonList(
|
|
|
+ new OmKeyLocationInfoGroup(0, locations)))
|
|
|
+ .setCreationTime(Time.now())
|
|
|
+ .setModificationTime(Time.now())
|
|
|
+ .setDataSize(0)
|
|
|
+ .setReplicationType(type)
|
|
|
+ .setReplicationFactor(factor)
|
|
|
+ .setFileEncryptionInfo(encInfo)
|
|
|
+ .build();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * OzoneFS api to creates an output stream for a file.
|
|
|
+ *
|
|
|
+ * @param args Key args
|
|
|
+ * @param isOverWrite if true existing file at the location will be
|
|
|
+ * overwritten
|
|
|
+ * @param isRecursive if true file would be created even if parent
|
|
|
+ * directories do not exist
|
|
|
+ * @throws OMException if given key is a directory
|
|
|
+ * if file exists and isOverwrite flag is false
|
|
|
+ * if an ancestor exists as a file
|
|
|
+ * if bucket does not exist
|
|
|
+ * @throws IOException if there is error in the db
|
|
|
+ * invalid arguments
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public OpenKeySession createFile(OmKeyArgs args, boolean isOverWrite,
|
|
|
+ boolean isRecursive) throws IOException {
|
|
|
+ Preconditions.checkNotNull(args, "Key args can not be null");
|
|
|
+ String volumeName = args.getVolumeName();
|
|
|
+ String bucketName = args.getBucketName();
|
|
|
+ String keyName = args.getKeyName();
|
|
|
+ OpenKeySession keySession;
|
|
|
+
|
|
|
+ try {
|
|
|
+ metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
+
|
|
|
+ OzoneFileStatus fileStatus;
|
|
|
+ try {
|
|
|
+ fileStatus = getFileStatus(args);
|
|
|
+ if (fileStatus.isDirectory()) {
|
|
|
+ throw new OMException("Can not write to directory: " + keyName,
|
|
|
+ ResultCodes.NOT_A_FILE);
|
|
|
+ } else if (fileStatus.isFile()) {
|
|
|
+ if (!isOverWrite) {
|
|
|
+ throw new OMException("File " + keyName + " already exists",
|
|
|
+ ResultCodes.FILE_ALREADY_EXISTS);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ } catch (OMException ex) {
|
|
|
+ if (ex.getResult() != ResultCodes.FILE_NOT_FOUND) {
|
|
|
+ throw ex;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ verifyNoFilesInPath(volumeName, bucketName,
|
|
|
+ Paths.get(keyName).getParent(), !isRecursive);
|
|
|
+ // TODO: Optimize call to openKey as keyInfo is already available in the
|
|
|
+ // filestatus. We can avoid some operations in openKey call.
|
|
|
+ keySession = openKey(args);
|
|
|
+ } finally {
|
|
|
+ metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
|
|
+ }
|
|
|
+
|
|
|
+ return keySession;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * OzoneFS api to lookup for a file.
|
|
|
+ *
|
|
|
+ * @param args Key args
|
|
|
+ * @throws OMException if given key is not found or it is not a file
|
|
|
+ * if bucket does not exist
|
|
|
+ * @throws IOException if there is error in the db
|
|
|
+ * invalid arguments
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public OmKeyInfo lookupFile(OmKeyArgs args) throws IOException {
|
|
|
+ Preconditions.checkNotNull(args, "Key args can not be null");
|
|
|
+ String volumeName = args.getVolumeName();
|
|
|
+ String bucketName = args.getBucketName();
|
|
|
+ String keyName = args.getKeyName();
|
|
|
+
|
|
|
+ try {
|
|
|
+ metadataManager.getLock().acquireBucketLock(volumeName, bucketName);
|
|
|
+ OzoneFileStatus fileStatus = getFileStatus(args);
|
|
|
+ if (fileStatus.isFile()) {
|
|
|
+ return fileStatus.getKeyInfo();
|
|
|
+ }
|
|
|
+ //if key is not of type file or if key is not found we throw an exception
|
|
|
+ } finally {
|
|
|
+ metadataManager.getLock().releaseBucketLock(volumeName, bucketName);
|
|
|
+ }
|
|
|
+
|
|
|
+ throw new OMException("Can not write to directory: " + keyName,
|
|
|
+ ResultCodes.NOT_A_FILE);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Verify that none of the parent path exists as file in the filesystem.
|
|
|
+ *
|
|
|
+ * @param volumeName Volume name
|
|
|
+ * @param bucketName Bucket name
|
|
|
+ * @param path Directory path. This is the absolute path of the
|
|
|
+ * directory for the ozone filesystem.
|
|
|
+ * @param directoryMustExist throws exception if true and given path does not
|
|
|
+ * exist as directory
|
|
|
+ * @throws OMException if ancestor exists as file in the filesystem
|
|
|
+ * if directoryMustExist flag is true and parent does
|
|
|
+ * not exist
|
|
|
+ * if bucket does not exist
|
|
|
+ * @throws IOException if there is error in the db
|
|
|
+ * invalid arguments
|
|
|
+ */
|
|
|
+ private void verifyNoFilesInPath(String volumeName, String bucketName,
|
|
|
+ Path path, boolean directoryMustExist) throws IOException {
|
|
|
+ OmKeyArgs.Builder argsBuilder = new OmKeyArgs.Builder()
|
|
|
+ .setVolumeName(volumeName)
|
|
|
+ .setBucketName(bucketName);
|
|
|
+ while (path != null) {
|
|
|
+ String keyName = path.toString();
|
|
|
+ try {
|
|
|
+ OzoneFileStatus fileStatus =
|
|
|
+ getFileStatus(argsBuilder.setKeyName(keyName).build());
|
|
|
+ if (fileStatus.isFile()) {
|
|
|
+ LOG.error("Unable to create directory (File already exists): volume: "
|
|
|
+ + volumeName + "bucket: " + bucketName + "key: " + keyName);
|
|
|
+ throw new OMException(
|
|
|
+ "Unable to create directory at : volume: " + volumeName
|
|
|
+ + "bucket: " + bucketName + "key: " + keyName,
|
|
|
+ ResultCodes.FILE_ALREADY_EXISTS);
|
|
|
+ } else if (fileStatus.isDirectory()) {
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ } catch (OMException ex) {
|
|
|
+ if (ex.getResult() != ResultCodes.FILE_NOT_FOUND) {
|
|
|
+ throw ex;
|
|
|
+ } else if (ex.getResult() == ResultCodes.FILE_NOT_FOUND) {
|
|
|
+ if (directoryMustExist) {
|
|
|
+ throw new OMException("Parent directory does not exist",
|
|
|
+ ex.getCause(), ResultCodes.DIRECTORY_NOT_FOUND);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ path = path.getParent();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private FileEncryptionInfo getFileEncryptionInfo(OmBucketInfo bucketInfo)
|
|
|
+ throws IOException {
|
|
|
+ FileEncryptionInfo encInfo = null;
|
|
|
+ BucketEncryptionKeyInfo ezInfo = bucketInfo.getEncryptionKeyInfo();
|
|
|
+ if (ezInfo != null) {
|
|
|
+ if (getKMSProvider() == null) {
|
|
|
+ throw new OMException("Invalid KMS provider, check configuration " +
|
|
|
+ CommonConfigurationKeys.HADOOP_SECURITY_KEY_PROVIDER_PATH,
|
|
|
+ OMException.ResultCodes.INVALID_KMS_PROVIDER);
|
|
|
+ }
|
|
|
+
|
|
|
+ final String ezKeyName = ezInfo.getKeyName();
|
|
|
+ EncryptedKeyVersion edek = generateEDEK(ezKeyName);
|
|
|
+ encInfo = new FileEncryptionInfo(ezInfo.getSuite(), ezInfo.getVersion(),
|
|
|
+ edek.getEncryptedKeyVersion().getMaterial(),
|
|
|
+ edek.getEncryptedKeyIv(),
|
|
|
+ ezKeyName, edek.getEncryptionKeyVersionName());
|
|
|
+ }
|
|
|
+ return encInfo;
|
|
|
+ }
|
|
|
+
|
|
|
private String addTrailingSlashIfNeeded(String key) {
|
|
|
if (StringUtils.isNotEmpty(key) && !key.endsWith(OZONE_URI_DELIMITER)) {
|
|
|
return key + OZONE_URI_DELIMITER;
|