|
@@ -32,6 +32,7 @@ import java.security.cert.CertificateException;
|
|
import java.util.Collection;
|
|
import java.util.Collection;
|
|
import java.util.Objects;
|
|
import java.util.Objects;
|
|
|
|
|
|
|
|
+import org.apache.commons.codec.digest.DigestUtils;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
import org.apache.hadoop.crypto.key.KeyProvider;
|
|
import org.apache.hadoop.crypto.key.KeyProvider;
|
|
@@ -72,6 +73,7 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
import org.apache.hadoop.ipc.RPC;
|
|
import org.apache.hadoop.ipc.Server;
|
|
import org.apache.hadoop.ipc.Server;
|
|
import org.apache.hadoop.ozone.OzoneAcl;
|
|
import org.apache.hadoop.ozone.OzoneAcl;
|
|
|
|
+import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneIllegalArgumentException;
|
|
import org.apache.hadoop.ozone.OzoneIllegalArgumentException;
|
|
import org.apache.hadoop.ozone.OzoneSecurityUtil;
|
|
import org.apache.hadoop.ozone.OzoneSecurityUtil;
|
|
import org.apache.hadoop.ozone.om.ha.OMFailoverProxyProvider;
|
|
import org.apache.hadoop.ozone.om.ha.OMFailoverProxyProvider;
|
|
@@ -102,7 +104,6 @@ import org.apache.hadoop.ozone.audit.AuditLoggerType;
|
|
import org.apache.hadoop.ozone.audit.AuditMessage;
|
|
import org.apache.hadoop.ozone.audit.AuditMessage;
|
|
import org.apache.hadoop.ozone.audit.Auditor;
|
|
import org.apache.hadoop.ozone.audit.Auditor;
|
|
import org.apache.hadoop.ozone.audit.OMAction;
|
|
import org.apache.hadoop.ozone.audit.OMAction;
|
|
-import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
|
import org.apache.hadoop.ozone.common.Storage.StorageState;
|
|
import org.apache.hadoop.ozone.common.Storage.StorageState;
|
|
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
|
import org.apache.hadoop.ozone.om.exceptions.OMException;
|
|
import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
|
|
import org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes;
|
|
@@ -130,6 +131,7 @@ import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
|
|
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
|
|
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType;
|
|
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
|
|
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType;
|
|
import org.apache.hadoop.ozone.security.acl.OzoneAccessAuthorizer;
|
|
import org.apache.hadoop.ozone.security.acl.OzoneAccessAuthorizer;
|
|
|
|
+import org.apache.hadoop.ozone.security.acl.OzoneNativeAuthorizer;
|
|
import org.apache.hadoop.ozone.security.acl.OzoneObj;
|
|
import org.apache.hadoop.ozone.security.acl.OzoneObj;
|
|
import org.apache.hadoop.ozone.security.acl.OzoneObj.StoreType;
|
|
import org.apache.hadoop.ozone.security.acl.OzoneObj.StoreType;
|
|
import org.apache.hadoop.ozone.security.acl.OzoneObj.ResourceType;
|
|
import org.apache.hadoop.ozone.security.acl.OzoneObj.ResourceType;
|
|
@@ -209,6 +211,8 @@ import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME_DEFAULT;
|
|
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_USER_MAX_VOLUME_DEFAULT;
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_AUTH_METHOD;
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_AUTH_METHOD;
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_REQUEST;
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.INVALID_REQUEST;
|
|
|
|
+import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.KEY_NOT_FOUND;
|
|
|
|
+import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ADMINISTRATORS_WILDCARD;
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKEN_ERROR_OTHER;
|
|
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.TOKEN_ERROR_OTHER;
|
|
import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_BUCKET_LOCK;
|
|
import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.S3_BUCKET_LOCK;
|
|
import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
|
|
import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.VOLUME_LOCK;
|
|
@@ -277,6 +281,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
private File omRatisSnapshotDir;
|
|
private File omRatisSnapshotDir;
|
|
private final File ratisSnapshotFile;
|
|
private final File ratisSnapshotFile;
|
|
private long snapshotIndex;
|
|
private long snapshotIndex;
|
|
|
|
+ private final Collection<String> ozAdmins;
|
|
|
|
|
|
private KeyProviderCryptoExtension kmsProvider = null;
|
|
private KeyProviderCryptoExtension kmsProvider = null;
|
|
private static String keyProviderUriKeyName =
|
|
private static String keyProviderUriKeyName =
|
|
@@ -341,7 +346,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT);
|
|
OMConfigKeys.OZONE_OM_RATIS_ENABLE_DEFAULT);
|
|
startRatisServer();
|
|
startRatisServer();
|
|
startRatisClient();
|
|
startRatisClient();
|
|
-
|
|
|
|
if (isRatisEnabled) {
|
|
if (isRatisEnabled) {
|
|
// Create Ratis storage dir
|
|
// Create Ratis storage dir
|
|
String omRatisDirectory = OmUtils.getOMRatisDirectory(configuration);
|
|
String omRatisDirectory = OmUtils.getOMRatisDirectory(configuration);
|
|
@@ -350,7 +354,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
" must be defined.");
|
|
" must be defined.");
|
|
}
|
|
}
|
|
OmUtils.createOMDir(omRatisDirectory);
|
|
OmUtils.createOMDir(omRatisDirectory);
|
|
-
|
|
|
|
// Create Ratis snapshot dir
|
|
// Create Ratis snapshot dir
|
|
omRatisSnapshotDir = OmUtils.createOMDir(
|
|
omRatisSnapshotDir = OmUtils.createOMDir(
|
|
OmUtils.getOMRatisSnapshotDirectory(configuration));
|
|
OmUtils.getOMRatisSnapshotDirectory(configuration));
|
|
@@ -367,9 +370,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
|
|
|
InetSocketAddress omNodeRpcAddr = omNodeDetails.getRpcAddress();
|
|
InetSocketAddress omNodeRpcAddr = omNodeDetails.getRpcAddress();
|
|
omRpcAddressTxt = new Text(omNodeDetails.getRpcAddressString());
|
|
omRpcAddressTxt = new Text(omNodeDetails.getRpcAddressString());
|
|
-
|
|
|
|
secConfig = new SecurityConfig(configuration);
|
|
secConfig = new SecurityConfig(configuration);
|
|
-
|
|
|
|
volumeManager = new VolumeManagerImpl(metadataManager, configuration);
|
|
volumeManager = new VolumeManagerImpl(metadataManager, configuration);
|
|
|
|
|
|
// Create the KMS Key Provider
|
|
// Create the KMS Key Provider
|
|
@@ -407,9 +408,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
this.scmClient = new ScmClient(scmBlockClient, scmContainerClient);
|
|
this.scmClient = new ScmClient(scmBlockClient, scmContainerClient);
|
|
keyManager = new KeyManagerImpl(scmClient, metadataManager,
|
|
keyManager = new KeyManagerImpl(scmClient, metadataManager,
|
|
configuration, omStorage.getOmId(), blockTokenMgr, getKmsProvider());
|
|
configuration, omStorage.getOmId(), blockTokenMgr, getKmsProvider());
|
|
-
|
|
|
|
prefixManager = new PrefixManagerImpl(metadataManager);
|
|
prefixManager = new PrefixManagerImpl(metadataManager);
|
|
-
|
|
|
|
shutdownHook = () -> {
|
|
shutdownHook = () -> {
|
|
saveOmMetrics();
|
|
saveOmMetrics();
|
|
};
|
|
};
|
|
@@ -419,9 +418,19 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
OZONE_ACL_ENABLED_DEFAULT);
|
|
OZONE_ACL_ENABLED_DEFAULT);
|
|
if (isAclEnabled) {
|
|
if (isAclEnabled) {
|
|
accessAuthorizer = getACLAuthorizerInstance(conf);
|
|
accessAuthorizer = getACLAuthorizerInstance(conf);
|
|
|
|
+ if (accessAuthorizer instanceof OzoneNativeAuthorizer) {
|
|
|
|
+ OzoneNativeAuthorizer authorizer =
|
|
|
|
+ (OzoneNativeAuthorizer) accessAuthorizer;
|
|
|
|
+ authorizer.setVolumeManager(volumeManager);
|
|
|
|
+ authorizer.setBucketManager(bucketManager);
|
|
|
|
+ authorizer.setKeyManager(keyManager);
|
|
|
|
+ authorizer.setPrefixManager(prefixManager);
|
|
|
|
+ }
|
|
} else {
|
|
} else {
|
|
accessAuthorizer = null;
|
|
accessAuthorizer = null;
|
|
}
|
|
}
|
|
|
|
+ ozAdmins = conf.getTrimmedStringCollection(OzoneConfigKeys
|
|
|
|
+ .OZONE_ADMINISTRATORS);
|
|
omMetaDir = OmUtils.getOmDbDir(configuration);
|
|
omMetaDir = OmUtils.getOmDbDir(configuration);
|
|
|
|
|
|
this.scmBlockSize = (long) conf
|
|
this.scmBlockSize = (long) conf
|
|
@@ -1676,8 +1685,14 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
public void createVolume(OmVolumeArgs args) throws IOException {
|
|
public void createVolume(OmVolumeArgs args) throws IOException {
|
|
try {
|
|
try {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.VOLUME, StoreType.OZONE,
|
|
|
|
- ACLType.CREATE, args.getVolume(), null, null);
|
|
|
|
|
|
+ if (!ozAdmins.contains(OZONE_ADMINISTRATORS_WILDCARD) &&
|
|
|
|
+ !ozAdmins.contains(ProtobufRpcEngine.Server.getRemoteUser()
|
|
|
|
+ .getUserName())) {
|
|
|
|
+ LOG.error("Only admin users are authorized to create " +
|
|
|
|
+ "Ozone volumes.");
|
|
|
|
+ throw new OMException("Only admin users are authorized to create " +
|
|
|
|
+ "Ozone volumes.", ResultCodes.PERMISSION_DENIED);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
metrics.incNumVolumeCreates();
|
|
metrics.incNumVolumeCreates();
|
|
volumeManager.createVolume(args);
|
|
volumeManager.createVolume(args);
|
|
@@ -2003,8 +2018,13 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
public List<OmVolumeArgs> listVolumeByUser(String userName, String prefix,
|
|
public List<OmVolumeArgs> listVolumeByUser(String userName, String prefix,
|
|
String prevKey, int maxKeys) throws IOException {
|
|
String prevKey, int maxKeys) throws IOException {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.VOLUME, StoreType.OZONE, ACLType.LIST, prefix,
|
|
|
|
- null, null);
|
|
|
|
|
|
+ UserGroupInformation remoteUserUgi = ProtobufRpcEngine.Server.
|
|
|
|
+ getRemoteUser();
|
|
|
|
+ if (remoteUserUgi == null) {
|
|
|
|
+ LOG.error("Rpc user UGI is null. Authorization failed.");
|
|
|
|
+ throw new OMException("Rpc user UGI is null. Authorization " +
|
|
|
|
+ "failed.", ResultCodes.PERMISSION_DENIED);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
boolean auditSuccess = true;
|
|
boolean auditSuccess = true;
|
|
Map<String, String> auditMap = new LinkedHashMap<>();
|
|
Map<String, String> auditMap = new LinkedHashMap<>();
|
|
@@ -2043,8 +2063,14 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
public List<OmVolumeArgs> listAllVolumes(String prefix, String prevKey, int
|
|
public List<OmVolumeArgs> listAllVolumes(String prefix, String prevKey, int
|
|
maxKeys) throws IOException {
|
|
maxKeys) throws IOException {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.VOLUME, StoreType.OZONE, ACLType.LIST, prefix,
|
|
|
|
- null, null);
|
|
|
|
|
|
+ if (!ozAdmins.contains(ProtobufRpcEngine.Server.
|
|
|
|
+ getRemoteUser().getUserName())
|
|
|
|
+ && !ozAdmins.contains(OZONE_ADMINISTRATORS_WILDCARD)) {
|
|
|
|
+ LOG.error("Only admin users are authorized to create " +
|
|
|
|
+ "Ozone volumes.");
|
|
|
|
+ throw new OMException("Only admin users are authorized to create " +
|
|
|
|
+ "Ozone volumes.", ResultCodes.PERMISSION_DENIED);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
boolean auditSuccess = true;
|
|
boolean auditSuccess = true;
|
|
Map<String, String> auditMap = new LinkedHashMap<>();
|
|
Map<String, String> auditMap = new LinkedHashMap<>();
|
|
@@ -2079,7 +2105,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
public void createBucket(OmBucketInfo bucketInfo) throws IOException {
|
|
public void createBucket(OmBucketInfo bucketInfo) throws IOException {
|
|
try {
|
|
try {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.BUCKET, StoreType.OZONE, ACLType.CREATE,
|
|
|
|
|
|
+ checkAcls(ResourceType.VOLUME, StoreType.OZONE, ACLType.CREATE,
|
|
bucketInfo.getVolumeName(), bucketInfo.getBucketName(), null);
|
|
bucketInfo.getVolumeName(), bucketInfo.getBucketName(), null);
|
|
}
|
|
}
|
|
metrics.incNumBucketCreates();
|
|
metrics.incNumBucketCreates();
|
|
@@ -2175,8 +2201,19 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
@Override
|
|
@Override
|
|
public OpenKeySession openKey(OmKeyArgs args) throws IOException {
|
|
public OpenKeySession openKey(OmKeyArgs args) throws IOException {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.READ,
|
|
|
|
- args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
|
|
+ try {
|
|
|
|
+ checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
+ args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
+ } catch (OMException ex) {
|
|
|
|
+ // For new keys key checkAccess call will fail as key doesn't exist.
|
|
|
|
+ // Check user access for bucket.
|
|
|
|
+ if (ex.getResult().equals(KEY_NOT_FOUND)) {
|
|
|
|
+ checkAcls(ResourceType.BUCKET, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
+ args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
+ } else {
|
|
|
|
+ throw ex;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
boolean auditSuccess = true;
|
|
boolean auditSuccess = true;
|
|
try {
|
|
try {
|
|
@@ -2245,8 +2282,19 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
public void commitKey(OmKeyArgs args, long clientID)
|
|
public void commitKey(OmKeyArgs args, long clientID)
|
|
throws IOException {
|
|
throws IOException {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
- args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
|
|
+ try {
|
|
|
|
+ checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
+ args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
+ } catch (OMException ex) {
|
|
|
|
+ // For new keys key checkAccess call will fail as key doesn't exist.
|
|
|
|
+ // Check user access for bucket.
|
|
|
|
+ if (ex.getResult().equals(KEY_NOT_FOUND)) {
|
|
|
|
+ checkAcls(ResourceType.BUCKET, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
+ args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
+ } else {
|
|
|
|
+ throw ex;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
Map<String, String> auditMap = (args == null) ? new LinkedHashMap<>() :
|
|
Map<String, String> auditMap = (args == null) ? new LinkedHashMap<>() :
|
|
args.toAuditMap();
|
|
args.toAuditMap();
|
|
@@ -2276,11 +2324,21 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID,
|
|
public OmKeyLocationInfo allocateBlock(OmKeyArgs args, long clientID,
|
|
- ExcludeList excludeList)
|
|
|
|
- throws IOException {
|
|
|
|
|
|
+ ExcludeList excludeList) throws IOException {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
- args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
|
|
+ try {
|
|
|
|
+ checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
+ args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
+ } catch (OMException ex) {
|
|
|
|
+ // For new keys key checkAccess call will fail as key doesn't exist.
|
|
|
|
+ // Check user access for bucket.
|
|
|
|
+ if (ex.getResult().equals(KEY_NOT_FOUND)) {
|
|
|
|
+ checkAcls(ResourceType.BUCKET, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
+ args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
+ } else {
|
|
|
|
+ throw ex;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
}
|
|
}
|
|
boolean auditSuccess = true;
|
|
boolean auditSuccess = true;
|
|
Map<String, String> auditMap = (args == null) ? new LinkedHashMap<>() :
|
|
Map<String, String> auditMap = (args == null) ? new LinkedHashMap<>() :
|
|
@@ -2414,8 +2472,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
|
|
public List<OmKeyInfo> listKeys(String volumeName, String bucketName,
|
|
String startKey, String keyPrefix, int maxKeys) throws IOException {
|
|
String startKey, String keyPrefix, int maxKeys) throws IOException {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.LIST, volumeName,
|
|
|
|
- bucketName, keyPrefix);
|
|
|
|
|
|
+ checkAcls(ResourceType.BUCKET,
|
|
|
|
+ StoreType.OZONE, ACLType.LIST, volumeName, bucketName, keyPrefix);
|
|
}
|
|
}
|
|
boolean auditSuccess = true;
|
|
boolean auditSuccess = true;
|
|
Map<String, String> auditMap = buildAuditMap(volumeName);
|
|
Map<String, String> auditMap = buildAuditMap(volumeName);
|
|
@@ -2637,10 +2695,6 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
boolean acquiredS3Lock = false;
|
|
boolean acquiredS3Lock = false;
|
|
boolean acquiredVolumeLock = false;
|
|
boolean acquiredVolumeLock = false;
|
|
try {
|
|
try {
|
|
- if(isAclEnabled) {
|
|
|
|
- checkAcls(ResourceType.BUCKET, StoreType.S3, ACLType.CREATE,
|
|
|
|
- null, s3BucketName, null);
|
|
|
|
- }
|
|
|
|
metrics.incNumBucketCreates();
|
|
metrics.incNumBucketCreates();
|
|
acquiredS3Lock = metadataManager.getLock().acquireLock(S3_BUCKET_LOCK,
|
|
acquiredS3Lock = metadataManager.getLock().acquireLock(S3_BUCKET_LOCK,
|
|
s3BucketName);
|
|
s3BucketName);
|
|
@@ -2684,8 +2738,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
public void deleteS3Bucket(String s3BucketName) throws IOException {
|
|
public void deleteS3Bucket(String s3BucketName) throws IOException {
|
|
try {
|
|
try {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.BUCKET, StoreType.S3, ACLType.DELETE, null,
|
|
|
|
- s3BucketName, null);
|
|
|
|
|
|
+ checkAcls(ResourceType.BUCKET, StoreType.S3, ACLType.DELETE,
|
|
|
|
+ getS3VolumeName(), s3BucketName, null);
|
|
}
|
|
}
|
|
metrics.incNumBucketDeletes();
|
|
metrics.incNumBucketDeletes();
|
|
s3BucketManager.deleteS3Bucket(s3BucketName);
|
|
s3BucketManager.deleteS3Bucket(s3BucketName);
|
|
@@ -2711,11 +2765,19 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
throws IOException {
|
|
throws IOException {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
checkAcls(ResourceType.BUCKET, StoreType.S3, ACLType.READ,
|
|
checkAcls(ResourceType.BUCKET, StoreType.S3, ACLType.READ,
|
|
- null, s3BucketName, null);
|
|
|
|
|
|
+ getS3VolumeName(), s3BucketName, null);
|
|
}
|
|
}
|
|
return s3BucketManager.getOzoneBucketMapping(s3BucketName);
|
|
return s3BucketManager.getOzoneBucketMapping(s3BucketName);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Helper function to return volume name for S3 users.
|
|
|
|
+ * */
|
|
|
|
+ private String getS3VolumeName() {
|
|
|
|
+ return s3BucketManager.formatOzoneVolumeName(DigestUtils.md5Hex(
|
|
|
|
+ ProtobufRpcEngine.Server.getRemoteUser().getUserName().toLowerCase()));
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public List<OmBucketInfo> listS3Buckets(String userName, String startKey,
|
|
public List<OmBucketInfo> listS3Buckets(String userName, String startKey,
|
|
String prefix, int maxNumOfBuckets)
|
|
String prefix, int maxNumOfBuckets)
|
|
@@ -2892,7 +2954,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
@Override
|
|
@Override
|
|
public OzoneFileStatus getFileStatus(OmKeyArgs args) throws IOException {
|
|
public OzoneFileStatus getFileStatus(OmKeyArgs args) throws IOException {
|
|
if (isAclEnabled) {
|
|
if (isAclEnabled) {
|
|
- checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.READ,
|
|
|
|
|
|
+ checkAcls(getResourceType(args), StoreType.OZONE, ACLType.READ,
|
|
args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
}
|
|
}
|
|
boolean auditSuccess = true;
|
|
boolean auditSuccess = true;
|
|
@@ -2915,10 +2977,17 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private ResourceType getResourceType(OmKeyArgs args) {
|
|
|
|
+ if (args.getKeyName() == null || args.getKeyName().length() == 0) {
|
|
|
|
+ return ResourceType.BUCKET;
|
|
|
|
+ }
|
|
|
|
+ return ResourceType.KEY;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public void createDirectory(OmKeyArgs args) throws IOException {
|
|
public void createDirectory(OmKeyArgs args) throws IOException {
|
|
if (isAclEnabled) {
|
|
if (isAclEnabled) {
|
|
- checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
|
|
+ checkAcls(ResourceType.BUCKET, StoreType.OZONE, ACLType.WRITE,
|
|
args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
}
|
|
}
|
|
boolean auditSuccess = true;
|
|
boolean auditSuccess = true;
|
|
@@ -2945,8 +3014,8 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
public OpenKeySession createFile(OmKeyArgs args, boolean overWrite,
|
|
public OpenKeySession createFile(OmKeyArgs args, boolean overWrite,
|
|
boolean recursive) throws IOException {
|
|
boolean recursive) throws IOException {
|
|
if (isAclEnabled) {
|
|
if (isAclEnabled) {
|
|
- checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
- args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
|
|
|
|
+ checkAcls(ResourceType.BUCKET, StoreType.OZONE, ACLType.WRITE,
|
|
|
|
+ args.getVolumeName(), args.getBucketName(), null);
|
|
}
|
|
}
|
|
boolean auditSuccess = true;
|
|
boolean auditSuccess = true;
|
|
try {
|
|
try {
|
|
@@ -2994,7 +3063,7 @@ public final class OzoneManager extends ServiceRuntimeInfoImpl
|
|
public List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
|
|
public List<OzoneFileStatus> listStatus(OmKeyArgs args, boolean recursive,
|
|
String startKey, long numEntries) throws IOException {
|
|
String startKey, long numEntries) throws IOException {
|
|
if(isAclEnabled) {
|
|
if(isAclEnabled) {
|
|
- checkAcls(ResourceType.KEY, StoreType.OZONE, ACLType.READ,
|
|
|
|
|
|
+ checkAcls(getResourceType(args), StoreType.OZONE, ACLType.READ,
|
|
args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
args.getVolumeName(), args.getBucketName(), args.getKeyName());
|
|
}
|
|
}
|
|
boolean auditSuccess = true;
|
|
boolean auditSuccess = true;
|