Skip to content
Closed
Show file tree
Hide file tree
Changes from 14 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -237,6 +237,7 @@ private OzoneConsts() {
public static final String VOLUME = "volume";
public static final String BUCKET = "bucket";
public static final String KEY = "key";
public static final String OPEN_KEY = "openKey";
public static final String QUOTA = "quota";
public static final String QUOTA_IN_BYTES = "quotaInBytes";
public static final String OBJECT_ID = "objectID";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,8 @@

import static org.apache.hadoop.hdds.HddsUtils.getHostNameFromConfigKeys;
import static org.apache.hadoop.hdds.HddsUtils.getPortNumberFromConfigKeys;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ACL_AUTHORIZER_CLASS_NATIVE;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_ADDRESS_KEY;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_BIND_HOST_DEFAULT;
import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTPS_ADDRESS_KEY;
Expand Down Expand Up @@ -525,4 +527,16 @@ public static RepeatedOmKeyInfo prepareKeyForDelete(OmKeyInfo keyInfo,

return repeatedOmKeyInfo;
}

/**
* Returns true if OzoneNativeAuthorizer is configured in the configuration.
* @param configuration ozone configuration
* @return true if OzoneNativeAuthorizer is configured in the configuration;
* else false.
*/
public static boolean isNativeAuthorizerEnabled(Configuration configuration) {
String authorizer = configuration.get(OZONE_ACL_AUTHORIZER_CLASS);
return authorizer != null &&
authorizer.equals(OZONE_ACL_AUTHORIZER_CLASS_NATIVE);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -340,7 +340,7 @@ private OMResponse submitRequest(OMRequest omRequest)
if (omResponse.hasLeaderOMNodeId() && omFailoverProxyProvider != null) {
String leaderOmId = omResponse.getLeaderOMNodeId();

// Failover to the OM node returned by OMReponse leaderOMNodeId if
// Failover to the OM node returned by OMResponse leaderOMNodeId if
// current proxy is not pointing to that node.
omFailoverProxyProvider.performFailoverIfRequired(leaderOmId);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ public static int getNoOfAcls() {

public static ACLType getAclTypeFromOrdinal(int ordinal) {
if (ordinal > length - 1 && ordinal > -1) {
throw new IllegalArgumentException("Ordinal greater than array lentgh" +
throw new IllegalArgumentException("Ordinal greater than array length" +
". ordinal:" + ordinal);
}
return vals[ordinal];
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ public enum ResourceType {
VOLUME(OzoneConsts.VOLUME),
BUCKET(OzoneConsts.BUCKET),
KEY(OzoneConsts.KEY),
OPEN_KEY(OzoneConsts.OPEN_KEY),
Copy link
Contributor

@xiaoyuyao xiaoyuyao Oct 11, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add some comment on why OPEN_KEY is needed as ozone object type?
Do we have the corresponding acl type semantics documented somewhere?

PREFIX(OzoneConsts.PREFIX);

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ public void testBucketCreationPermissionDenied() throws Exception {
() -> volume.createBucket(bucketName));

assertTrue(logCapturer.getOutput()
.contains("doesn't have CREATE permission to access volume"));
.contains("doesn't have CREATE permission to access bucket"));
}

@Test
Expand All @@ -133,8 +133,8 @@ public void testFailureInKeyOp() throws Exception {

OzoneTestUtils.expectOmException(ResultCodes.PERMISSION_DENIED,
() -> TestDataUtil.createKey(bucket, "testKey", "testcontent"));
assertTrue(logCapturer.getOutput().contains("doesn't have WRITE " +
"permission to access bucket"));
assertTrue(logCapturer.getOutput().contains("doesn't have CREATE " +
"permission to access key"));
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@
import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType.USER;
import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLIdentityType.WORLD;
import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.ALL;
import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.CREATE;
import static org.apache.hadoop.ozone.security.acl.IAccessAuthorizer.ACLType.NONE;
import static org.apache.hadoop.ozone.security.acl.OzoneObj.ResourceType.BUCKET;
import static org.apache.hadoop.ozone.security.acl.OzoneObj.ResourceType.KEY;
Expand Down Expand Up @@ -362,6 +363,7 @@ private void resetAclsAndValidateAccess(OzoneObj obj,

aclsToBeAdded.remove(NONE);
aclsToBeAdded.remove(ALL);
aclsToBeAdded.remove(CREATE);

// Fetch acls again.
for (ACLType a2 : aclsToBeAdded) {
Expand Down Expand Up @@ -410,7 +412,7 @@ private void resetAclsAndValidateAccess(OzoneObj obj,
builder.setAclRights(a2).build()));
aclsToBeValidated.remove(a2);
for (ACLType a3 : aclsToBeValidated) {
if (!a3.equals(a1) && !a3.equals(a2)) {
if (!a3.equals(a1) && !a3.equals(a2) && !a3.equals(CREATE)) {
assertFalse("User shouldn't have right " + a3 + ". " +
"Current acl rights for user:" + a1 + "," + a2,
nativeAuthorizer.checkAccess(obj,
Expand Down Expand Up @@ -462,6 +464,7 @@ private void validateNone(OzoneObj obj, RequestContext.Builder
builder) throws OMException {
List<ACLType> allAcls = new ArrayList<>(Arrays.asList(ACLType.values()));
allAcls.remove(NONE);
allAcls.remove(CREATE);
for (ACLType a : allAcls) {
assertFalse("User shouldn't have right " + a + ".",
nativeAuthorizer.checkAccess(obj, builder.setAclRights(a).build()));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -123,6 +123,7 @@
import static org.apache.hadoop.ozone.om.exceptions.OMException.ResultCodes.VOLUME_NOT_FOUND;
import static org.apache.hadoop.ozone.om.lock.OzoneManagerLock.Resource.BUCKET_LOCK;
import static org.apache.hadoop.ozone.security.acl.OzoneObj.ResourceType.KEY;
import static org.apache.hadoop.ozone.security.acl.OzoneObj.ResourceType.OPEN_KEY;
import static org.apache.hadoop.util.Time.monotonicNow;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -1656,8 +1657,13 @@ public boolean checkAccess(OzoneObj ozObject, RequestContext context)
validateBucket(volume, bucket);
OmKeyInfo keyInfo = null;
try {
OzoneFileStatus fileStatus = getFileStatus(args);
keyInfo = fileStatus.getKeyInfo();
if (ozObject.getResourceType() == OPEN_KEY) {
keyInfo = metadataManager.getOpenKeyTable().get(objectKey);
} else {
OzoneFileStatus fileStatus = getFileStatus(args);
keyInfo = fileStatus.getKeyInfo();
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We Should remove lines from 1675-1677 And 1680-1683. As we have handled using openKey. So, we should throw an exception in CatchBlock

if (keyInfo == null) {
// the key does not exist, but it is a parent "dir" of some key
// let access be determined based on volume/bucket/prefix ACL
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -143,7 +143,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
try {
// check Acl
if (ozoneManager.getAclsEnabled()) {
checkAcls(ozoneManager, OzoneObj.ResourceType.VOLUME,
checkAcls(ozoneManager, OzoneObj.ResourceType.BUCKET,
OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.CREATE,
volumeName, bucketName, null);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -95,7 +95,7 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
// check Acl
if (ozoneManager.getAclsEnabled()) {
checkAcls(ozoneManager, OzoneObj.ResourceType.BUCKET,
OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.WRITE,
OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.DELETE,
volumeName, bucketName, null);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@
import org.apache.hadoop.ozone.om.helpers.OzoneAclUtil;
import org.apache.hadoop.ozone.om.helpers.OzoneFSUtils;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -127,7 +129,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
OMClientResponse omClientResponse = null;
try {
// check Acl
checkBucketAcls(ozoneManager, volumeName, bucketName, keyName);
checkKeyAcls(ozoneManager, volumeName, bucketName, keyName,
IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);

// Check if this is the root of the filesystem.
if (keyName.length() == 0) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,8 @@
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -177,7 +179,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
OMClientResponse omClientResponse = null;
try {
// check Acl
checkBucketAcls(ozoneManager, volumeName, bucketName, keyName);
checkKeyAcls(ozoneManager, volumeName, bucketName, keyName,
IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);

// acquire lock
acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,11 @@

import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
Expand Down Expand Up @@ -169,7 +173,22 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
OmKeyInfo omKeyInfo = null;
try {
// check Acl
checkBucketAcls(ozoneManager, volumeName, bucketName, keyName);
// Native authorizer requires client id as part of keyname to check
// write ACL on key. Add client id to key name if ozone native
// authorizer is configured.
Configuration config = ozoneManager.getConfiguration();
if (OmUtils.isNativeAuthorizerEnabled(config)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think here, instead of reading this from config every time for each operation, as this is in hot path, we can make add an method in OzoneManager, to return true/false depending on Authorizer.

https://github.com/apache/hadoop/blob/trunk/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManager.java#L474
We can declare a variable boolean isNativeAuthorizer in OzoneManager. And set to true. And then we can expose a method to return this variable value through isNativeAuthorizer()

String keyNameForAclCheck =
keyName + "/" + allocateBlockRequest.getClientID();
// During allocate block request, it is possible that key is
// not present in the key table and hence setting the resource type
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we move this to a common method, as it is being used in KeyCommitRequest also.

// to OPEN_KEY to check the openKeyTable.
checkKeyAcls(ozoneManager, volumeName, bucketName, keyNameForAclCheck,
IAccessAuthorizer.ACLType.WRITE, OzoneObj.ResourceType.OPEN_KEY);
} else {
checkKeyAcls(ozoneManager, volumeName, bucketName, keyName,
IAccessAuthorizer.ACLType.WRITE, OzoneObj.ResourceType.KEY);
}

OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
validateBucketAndVolume(omMetadataManager, volumeName,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,11 @@

import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ozone.OmUtils;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -111,11 +115,27 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
IOException exception = null;
OmKeyInfo omKeyInfo = null;
OMClientResponse omClientResponse = null;
boolean bucketLockAcquired = false;

OMMetadataManager omMetadataManager = ozoneManager.getMetadataManager();
try {
// check Acl
checkBucketAcls(ozoneManager, volumeName, bucketName, keyName);
// Native authorizer requires client id as part of keyname to check
// write ACL on key. Add client id to key name if ozone native
// authorizer is configured.
Configuration config = ozoneManager.getConfiguration();
if (OmUtils.isNativeAuthorizerEnabled(config)) {
String keyNameForAclCheck =
keyName + "/" + commitKeyRequest.getClientID();
// During key commit request, it is possible that key is
// not present in the key table and hence setting the resource type
// to OPEN_KEY to check the openKeyTable.
checkKeyAcls(ozoneManager, volumeName, bucketName, keyNameForAclCheck,
IAccessAuthorizer.ACLType.WRITE, OzoneObj.ResourceType.OPEN_KEY);
} else {
checkKeyAcls(ozoneManager, volumeName, bucketName, keyName,
IAccessAuthorizer.ACLType.WRITE, OzoneObj.ResourceType.KEY);
}

List<OmKeyLocationInfo> locationInfoList = commitKeyArgs
.getKeyLocationsList().stream()
Expand All @@ -127,8 +147,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
String dbOpenKey = omMetadataManager.getOpenKey(volumeName, bucketName,
keyName, commitKeyRequest.getClientID());

omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK, volumeName,
bucketName);
bucketLockAcquired = omMetadataManager.getLock()
.acquireWriteLock(BUCKET_LOCK, volumeName, bucketName);

validateBucketAndVolume(omMetadataManager, volumeName, bucketName);
omKeyInfo = omMetadataManager.getOpenKeyTable().get(dbOpenKey);
Expand Down Expand Up @@ -166,8 +186,11 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
ozoneManagerDoubleBufferHelper.add(omClientResponse,
transactionLogIndex));
}
omMetadataManager.getLock().releaseWriteLock(BUCKET_LOCK, volumeName,
bucketName);

if(bucketLockAcquired) {
omMetadataManager.getLock().releaseWriteLock(BUCKET_LOCK, volumeName,
bucketName);
}
}

// Performing audit logging outside of the lock.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,8 @@
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -162,7 +164,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
OMClientResponse omClientResponse = null;
try {
// check Acl
checkBucketAcls(ozoneManager, volumeName, bucketName, keyName);
checkKeyAcls(ozoneManager, volumeName, bucketName, keyName,
IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);

acquireLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
volumeName, bucketName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@

import com.google.common.base.Optional;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -109,7 +111,8 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
OMClientResponse omClientResponse = null;
try {
// check Acl
checkKeyAcls(ozoneManager, volumeName, bucketName, keyName);
checkKeyAcls(ozoneManager, volumeName, bucketName, keyName,
IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);

String objectKey = omMetadataManager.getOzoneKey(
volumeName, bucketName, keyName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import org.apache.hadoop.ozone.om.ratis.utils.OzoneManagerDoubleBufferHelper;
import org.apache.hadoop.ozone.security.acl.IAccessAuthorizer;
import org.apache.hadoop.ozone.security.acl.OzoneObj;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand Down Expand Up @@ -117,8 +119,12 @@ public OMClientResponse validateAndUpdateCache(OzoneManager ozoneManager,
throw new OMException("Key name is empty",
OMException.ResultCodes.INVALID_KEY_NAME);
}
// check Acl
checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName);
// check Acls to see if user has access to perform delete operation on
// old key and create operation on new key
checkKeyAcls(ozoneManager, volumeName, bucketName, fromKeyName,
IAccessAuthorizer.ACLType.DELETE, OzoneObj.ResourceType.KEY);
checkKeyAcls(ozoneManager, volumeName, bucketName, toKeyName,
IAccessAuthorizer.ACLType.CREATE, OzoneObj.ResourceType.KEY);

acquiredLock = omMetadataManager.getLock().acquireWriteLock(BUCKET_LOCK,
volumeName, bucketName);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -507,10 +507,11 @@ private OMClientResponse createKeyErrorResponse(@Nonnull OMMetrics omMetrics,
* @throws IOException
*/
protected void checkBucketAcls(OzoneManager ozoneManager, String volume,
String bucket, String key) throws IOException {
String bucket, String key, IAccessAuthorizer.ACLType aclType)
throws IOException {
if (ozoneManager.getAclsEnabled()) {
checkAcls(ozoneManager, OzoneObj.ResourceType.BUCKET,
OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.WRITE,
OzoneObj.StoreType.OZONE, aclType,
volume, bucket, key);
}
}
Expand All @@ -525,10 +526,11 @@ protected void checkBucketAcls(OzoneManager ozoneManager, String volume,
* @throws IOException
*/
protected void checkKeyAcls(OzoneManager ozoneManager, String volume,
String bucket, String key) throws IOException {
String bucket, String key, IAccessAuthorizer.ACLType aclType,
OzoneObj.ResourceType resourceType)
throws IOException {
if (ozoneManager.getAclsEnabled()) {
checkAcls(ozoneManager, OzoneObj.ResourceType.KEY,
OzoneObj.StoreType.OZONE, IAccessAuthorizer.ACLType.WRITE,
checkAcls(ozoneManager, resourceType, OzoneObj.StoreType.OZONE, aclType,
volume, bucket, key);
}
}
Expand Down
Loading