|
@@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.om;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
import java.util.List;
|
|
|
+import java.util.Objects;
|
|
|
|
|
|
import org.apache.hadoop.crypto.CipherSuite;
|
|
|
import org.apache.hadoop.crypto.CryptoProtocolVersion;
|
|
@@ -30,6 +31,8 @@ import org.apache.hadoop.ozone.om.exceptions.OMException;
|
|
|
import org.apache.hadoop.ozone.om.helpers.BucketEncryptionKeyInfo;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmBucketArgs;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmBucketInfo;
|
|
|
+import org.apache.hadoop.ozone.security.acl.OzoneObj;
|
|
|
+import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.Time;
|
|
|
|
|
|
import com.google.common.base.Preconditions;
|
|
@@ -37,6 +40,8 @@ import org.iq80.leveldb.DBException;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.BUCKET_NOT_FOUND;
|
|
|
+
|
|
|
/**
|
|
|
* OM bucket manager.
|
|
|
*/
|
|
@@ -207,7 +212,7 @@ public class BucketManagerImpl implements BucketManager {
|
|
|
LOG.debug("bucket: {} not found in volume: {}.", bucketName,
|
|
|
volumeName);
|
|
|
throw new OMException("Bucket not found",
|
|
|
- OMException.ResultCodes.BUCKET_NOT_FOUND);
|
|
|
+ BUCKET_NOT_FOUND);
|
|
|
}
|
|
|
return value;
|
|
|
} catch (IOException | DBException ex) {
|
|
@@ -241,7 +246,7 @@ public class BucketManagerImpl implements BucketManager {
|
|
|
if (oldBucketInfo == null) {
|
|
|
LOG.debug("bucket: {} not found ", bucketName);
|
|
|
throw new OMException("Bucket doesn't exist",
|
|
|
- OMException.ResultCodes.BUCKET_NOT_FOUND);
|
|
|
+ BUCKET_NOT_FOUND);
|
|
|
}
|
|
|
OmBucketInfo.Builder bucketInfoBuilder = OmBucketInfo.newBuilder();
|
|
|
bucketInfoBuilder.setVolumeName(oldBucketInfo.getVolumeName())
|
|
@@ -333,7 +338,7 @@ public class BucketManagerImpl implements BucketManager {
|
|
|
if (metadataManager.getBucketTable().get(bucketKey) == null) {
|
|
|
LOG.debug("bucket: {} not found ", bucketName);
|
|
|
throw new OMException("Bucket doesn't exist",
|
|
|
- OMException.ResultCodes.BUCKET_NOT_FOUND);
|
|
|
+ BUCKET_NOT_FOUND);
|
|
|
}
|
|
|
//Check if bucket is empty
|
|
|
if (!metadataManager.isBucketEmpty(volumeName, bucketName)) {
|
|
@@ -370,4 +375,235 @@ public class BucketManagerImpl implements BucketManager {
|
|
|
volumeName, startBucket, bucketPrefix, maxNumOfBuckets);
|
|
|
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Add acl for Ozone object. Return true if acl is added successfully else
|
|
|
+ * false.
|
|
|
+ *
|
|
|
+ * @param obj Ozone object for which acl should be added.
|
|
|
+ * @param acl ozone acl top be added.
|
|
|
+ * @throws IOException if there is error.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public boolean addAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
|
|
|
+ Objects.requireNonNull(obj);
|
|
|
+ Objects.requireNonNull(acl);
|
|
|
+ if (!obj.getResourceType().equals(OzoneObj.ResourceType.BUCKET)) {
|
|
|
+ throw new IllegalArgumentException("Unexpected argument passed to " +
|
|
|
+ "BucketManager. OzoneObj type:" + obj.getResourceType());
|
|
|
+ }
|
|
|
+ String volume = obj.getVolumeName();
|
|
|
+ String bucket = obj.getBucketName();
|
|
|
+ metadataManager.getLock().acquireBucketLock(volume, bucket);
|
|
|
+ try {
|
|
|
+ String dbBucketKey = metadataManager.getBucketKey(volume, bucket);
|
|
|
+ OmBucketInfo bucketInfo =
|
|
|
+ metadataManager.getBucketTable().get(dbBucketKey);
|
|
|
+ if (bucketInfo == null) {
|
|
|
+ LOG.debug("Bucket:{}/{} does not exist", volume, bucket);
|
|
|
+ throw new OMException("Bucket " + bucket + " is not found",
|
|
|
+ BUCKET_NOT_FOUND);
|
|
|
+ }
|
|
|
+ List<OzoneAcl> list = bucketInfo.getAcls();
|
|
|
+ if(!validateAddAcl(acl, list)) {
|
|
|
+ // New acl can't be added as it is not consistent with existing ACLs.
|
|
|
+ LOG.info("New acl:{} can't be added as it is not consistent with " +
|
|
|
+ "existing ACLs:{}.", acl, StringUtils.join(",", list));
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ list.add(acl);
|
|
|
+ OmBucketInfo updatedBucket = OmBucketInfo.newBuilder()
|
|
|
+ .setVolumeName(bucketInfo.getVolumeName())
|
|
|
+ .setBucketName(bucketInfo.getBucketName())
|
|
|
+ .setStorageType(bucketInfo.getStorageType())
|
|
|
+ .setIsVersionEnabled(bucketInfo.getIsVersionEnabled())
|
|
|
+ .setCreationTime(bucketInfo.getCreationTime())
|
|
|
+ .setBucketEncryptionKey(bucketInfo.getEncryptionKeyInfo())
|
|
|
+ .addAllMetadata(bucketInfo.getMetadata())
|
|
|
+ .setAcls(list)
|
|
|
+ .build();
|
|
|
+ // TODO:HDDS-1619 OM HA changes required for all acl operations.
|
|
|
+
|
|
|
+ metadataManager.getBucketTable().put(dbBucketKey, updatedBucket);
|
|
|
+ } catch (IOException ex) {
|
|
|
+ if (!(ex instanceof OMException)) {
|
|
|
+ LOG.error("Add acl operation failed for bucket:{}/{} acl:{}",
|
|
|
+ volume, bucket, acl, ex);
|
|
|
+ }
|
|
|
+ throw ex;
|
|
|
+ } finally {
|
|
|
+ metadataManager.getLock().releaseBucketLock(volume, bucket);
|
|
|
+ }
|
|
|
+
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Remove acl for Ozone object. Return true if acl is removed successfully
|
|
|
+ * else false.
|
|
|
+ *
|
|
|
+ * @param obj Ozone object.
|
|
|
+ * @param acl Ozone acl to be removed.
|
|
|
+ * @throws IOException if there is error.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public boolean removeAcl(OzoneObj obj, OzoneAcl acl) throws IOException {
|
|
|
+ Objects.requireNonNull(obj);
|
|
|
+ Objects.requireNonNull(acl);
|
|
|
+ if (!obj.getResourceType().equals(OzoneObj.ResourceType.BUCKET)) {
|
|
|
+ throw new IllegalArgumentException("Unexpected argument passed to " +
|
|
|
+ "BucketManager. OzoneObj type:" + obj.getResourceType());
|
|
|
+ }
|
|
|
+ String volume = obj.getVolumeName();
|
|
|
+ String bucket = obj.getBucketName();
|
|
|
+ metadataManager.getLock().acquireBucketLock(volume, bucket);
|
|
|
+ try {
|
|
|
+ String dbBucketKey = metadataManager.getBucketKey(volume, bucket);
|
|
|
+ OmBucketInfo bucketInfo =
|
|
|
+ metadataManager.getBucketTable().get(dbBucketKey);
|
|
|
+ if (bucketInfo == null) {
|
|
|
+ LOG.debug("Bucket:{}/{} does not exist", volume, bucket);
|
|
|
+ throw new OMException("Bucket " + bucket + " is not found",
|
|
|
+ BUCKET_NOT_FOUND);
|
|
|
+ }
|
|
|
+ List<OzoneAcl> list = bucketInfo.getAcls();
|
|
|
+ if (!list.contains(acl)) {
|
|
|
+ // Return false if acl doesn't exist in current ACLs.
|
|
|
+ LOG.info("Acl:{} not found in existing ACLs:{}.", acl,
|
|
|
+ StringUtils.join(",", list));
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+ list.remove(acl);
|
|
|
+ OmBucketInfo updatedBucket = OmBucketInfo.newBuilder()
|
|
|
+ .setVolumeName(bucketInfo.getVolumeName())
|
|
|
+ .setBucketName(bucketInfo.getBucketName())
|
|
|
+ .setStorageType(bucketInfo.getStorageType())
|
|
|
+ .setIsVersionEnabled(bucketInfo.getIsVersionEnabled())
|
|
|
+ .setCreationTime(bucketInfo.getCreationTime())
|
|
|
+ .setBucketEncryptionKey(bucketInfo.getEncryptionKeyInfo())
|
|
|
+ .addAllMetadata(bucketInfo.getMetadata())
|
|
|
+ .setAcls(list)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ metadataManager.getBucketTable().put(dbBucketKey, updatedBucket);
|
|
|
+ } catch (IOException ex) {
|
|
|
+ if (!(ex instanceof OMException)) {
|
|
|
+ LOG.error("Remove acl operation failed for bucket:{}/{} acl:{}",
|
|
|
+ volume, bucket, acl, ex);
|
|
|
+ }
|
|
|
+ throw ex;
|
|
|
+ } finally {
|
|
|
+ metadataManager.getLock().releaseBucketLock(volume, bucket);
|
|
|
+ }
|
|
|
+
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Acls to be set for given Ozone object. This operations reset ACL for given
|
|
|
+ * object to list of ACLs provided in argument.
|
|
|
+ *
|
|
|
+ * @param obj Ozone object.
|
|
|
+ * @param acls List of acls.
|
|
|
+ * @throws IOException if there is error.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public boolean setAcl(OzoneObj obj, List<OzoneAcl> acls) throws IOException {
|
|
|
+ Objects.requireNonNull(obj);
|
|
|
+ Objects.requireNonNull(acls);
|
|
|
+ if (!obj.getResourceType().equals(OzoneObj.ResourceType.BUCKET)) {
|
|
|
+ throw new IllegalArgumentException("Unexpected argument passed to " +
|
|
|
+ "BucketManager. OzoneObj type:" + obj.getResourceType());
|
|
|
+ }
|
|
|
+ String volume = obj.getVolumeName();
|
|
|
+ String bucket = obj.getBucketName();
|
|
|
+ metadataManager.getLock().acquireBucketLock(volume, bucket);
|
|
|
+ try {
|
|
|
+ String dbBucketKey = metadataManager.getBucketKey(volume, bucket);
|
|
|
+ OmBucketInfo bucketInfo =
|
|
|
+ metadataManager.getBucketTable().get(dbBucketKey);
|
|
|
+ if (bucketInfo == null) {
|
|
|
+ LOG.debug("Bucket:{}/{} does not exist", volume, bucket);
|
|
|
+ throw new OMException("Bucket " + bucket + " is not found",
|
|
|
+ BUCKET_NOT_FOUND);
|
|
|
+ }
|
|
|
+ OmBucketInfo updatedBucket = OmBucketInfo.newBuilder()
|
|
|
+ .setVolumeName(bucketInfo.getVolumeName())
|
|
|
+ .setBucketName(bucketInfo.getBucketName())
|
|
|
+ .setStorageType(bucketInfo.getStorageType())
|
|
|
+ .setIsVersionEnabled(bucketInfo.getIsVersionEnabled())
|
|
|
+ .setCreationTime(bucketInfo.getCreationTime())
|
|
|
+ .setBucketEncryptionKey(bucketInfo.getEncryptionKeyInfo())
|
|
|
+ .addAllMetadata(bucketInfo.getMetadata())
|
|
|
+ .setAcls(acls)
|
|
|
+ .build();
|
|
|
+
|
|
|
+ metadataManager.getBucketTable().put(dbBucketKey, updatedBucket);
|
|
|
+ } catch (IOException ex) {
|
|
|
+ if (!(ex instanceof OMException)) {
|
|
|
+ LOG.error("Set acl operation failed for bucket:{}/{} acl:{}",
|
|
|
+ volume, bucket, StringUtils.join(",", acls), ex);
|
|
|
+ }
|
|
|
+ throw ex;
|
|
|
+ } finally {
|
|
|
+ metadataManager.getLock().releaseBucketLock(volume, bucket);
|
|
|
+ }
|
|
|
+
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Validates if a new acl addition is consistent with current ACL list.
|
|
|
+ * @param newAcl new acl to be added.
|
|
|
+ * @param currentAcls list of acls.
|
|
|
+ *
|
|
|
+ * @return true if newAcl addition to existing acls is valid, else false.
|
|
|
+ * */
|
|
|
+ private boolean validateAddAcl(OzoneAcl newAcl, List<OzoneAcl> currentAcls) {
|
|
|
+
|
|
|
+ // Check 1: Check for duplicate.
|
|
|
+ if(currentAcls.contains(newAcl)) {
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Returns list of ACLs for given Ozone object.
|
|
|
+ *
|
|
|
+ * @param obj Ozone object.
|
|
|
+ * @throws IOException if there is error.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public List<OzoneAcl> getAcl(OzoneObj obj) throws IOException {
|
|
|
+ Objects.requireNonNull(obj);
|
|
|
+
|
|
|
+ if (!obj.getResourceType().equals(OzoneObj.ResourceType.BUCKET)) {
|
|
|
+ throw new IllegalArgumentException("Unexpected argument passed to " +
|
|
|
+ "BucketManager. OzoneObj type:" + obj.getResourceType());
|
|
|
+ }
|
|
|
+ String volume = obj.getVolumeName();
|
|
|
+ String bucket = obj.getBucketName();
|
|
|
+ metadataManager.getLock().acquireBucketLock(volume, bucket);
|
|
|
+ try {
|
|
|
+ String dbBucketKey = metadataManager.getBucketKey(volume, bucket);
|
|
|
+ OmBucketInfo bucketInfo =
|
|
|
+ metadataManager.getBucketTable().get(dbBucketKey);
|
|
|
+ if (bucketInfo == null) {
|
|
|
+ LOG.debug("Bucket:{}/{} does not exist", volume, bucket);
|
|
|
+ throw new OMException("Bucket " + bucket + " is not found",
|
|
|
+ BUCKET_NOT_FOUND);
|
|
|
+ }
|
|
|
+ return bucketInfo.getAcls();
|
|
|
+ } catch (IOException ex) {
|
|
|
+ if (!(ex instanceof OMException)) {
|
|
|
+ LOG.error("Get acl operation failed for bucket:{}/{} acl:{}",
|
|
|
+ volume, bucket, ex);
|
|
|
+ }
|
|
|
+ throw ex;
|
|
|
+ } finally {
|
|
|
+ metadataManager.getLock().releaseBucketLock(volume, bucket);
|
|
|
+ }
|
|
|
+ }
|
|
|
}
|