From 042766f1cc23564df92c5f4a024ac1250fdc5b60 Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Thu, 31 Jan 2019 13:17:59 -0600 Subject: [PATCH 01/24] Support LDAP authentication/authorization --- .../extensions-core/druid-basic-security.md | 129 +++- .../security/basic/BasicAuthDBConfig.java | 112 +++- .../druid/security/basic/BasicAuthUtils.java | 122 +++- .../BasicSecurityAuthenticationException.java | 38 ++ .../basic/BasicSecurityDruidModule.java | 61 +- .../basic/BasicSecuritySSLSocketFactory.java | 136 ++++ .../security/basic/CommonCacheNotifier.java | 84 ++- .../BasicAuthenticatorUserPrincipal.java | 137 ++++ .../BasicHTTPAuthenticator.java | 114 ++-- .../authentication/BasicHTTPEscalator.java | 116 +++- .../cache/BasicAuthenticatorCacheManager.java | 22 +- .../BasicAuthenticatorCacheNotifier.java | 10 +- ...inatorBasicAuthenticatorCacheNotifier.java | 32 +- ...PollingBasicAuthenticatorCacheManager.java | 113 +++- ...PollingBasicAuthenticatorCacheManager.java | 14 +- ...icAuthenticatorMetadataStorageUpdater.java | 9 + ...icAuthenticatorMetadataStorageUpdater.java | 221 +++++- .../endpoint/BasicAuthenticatorResource.java | 80 ++- .../BasicAuthenticatorResourceHandler.java | 11 +- ...atorBasicAuthenticatorResourceHandler.java | 69 +- ...aultBasicAuthenticatorResourceHandler.java | 46 +- .../entity/BasicAuthConfig.java | 108 +++ .../BasicAuthenticatorConfigBundle.java | 51 ++ .../validator/CredentialsValidator.java | 39 ++ .../validator/DBCredentialsValidator.java | 88 +++ .../validator/LDAPCredentialsValidator.java | 422 ++++++++++++ .../BasicRoleBasedAuthorizer.java | 108 ++- .../db/cache/BasicAuthorizerCacheManager.java | 28 +- .../cache/BasicAuthorizerCacheNotifier.java | 12 +- ...ordinatorBasicAuthorizerCacheNotifier.java | 29 +- ...torPollingBasicAuthorizerCacheManager.java | 181 ++++- ...agePollingBasicAuthorizerCacheManager.java | 21 +- ...BasicAuthorizerMetadataStorageUpdater.java | 17 +- ...BasicAuthorizerMetadataStorageUpdater.java | 634 +++++++++++++++--- .../endpoint/BasicAuthorizerResource.java | 205 +++++- .../BasicAuthorizerResourceHandler.java | 23 +- ...dinatorBasicAuthorizerResourceHandler.java | 267 +++++++- ...DefaultBasicAuthorizerResourceHandler.java | 80 ++- .../entity/BasicAuthorizerGroupMapping.java | 97 +++ .../BasicAuthorizerGroupMappingFull.java | 96 +++ .../BasicAuthorizerGroupMappingMapBundle.java | 53 ++ .../entity/BasicAuthorizerRoleFull.java | 13 + .../entity/GroupMappingAndRoleMap.java | 56 ++ .../db/cache/BasicEscalatorCacheManager.java | 43 ++ .../db/cache/BasicEscalatorCacheNotifier.java | 33 + ...oordinatorBasicEscalatorCacheNotifier.java | 130 ++++ ...atorPollingBasicEscalatorCacheManager.java | 210 ++++++ ...ragePollingBasicEscalatorCacheManager.java | 57 ++ .../BasicEscalatorMetadataStorageUpdater.java | 42 ++ ...rBasicEscalatorMetadataStorageUpdater.java | 277 ++++++++ .../endpoint/BasicEscalatorResource.java | 158 +++++ .../BasicEscalatorResourceHandler.java | 47 ++ ...rdinatorBasicEscalatorResourceHandler.java | 118 ++++ .../DefaultBasicEscalatorResourceHandler.java | 83 +++ .../entity/BasicEscalatorCredential.java | 75 +++ .../BasicEscalatorCredentialBundle.java | 51 ++ .../BasicHTTPAuthenticatorTest.java | 205 +++++- ...thenticatorMetadataStorageUpdaterTest.java | 93 ++- ...dinatorBasicAuthenticatorResourceTest.java | 245 ++++++- .../NoopBasicAuthenticatorCacheNotifier.java | 7 +- .../validator/DBCredentialsValidatorTest.java | 165 +++++ .../BasicRoleBasedAuthorizerTest.java | 225 ++++++- ...cAuthorizerMetadataStorageUpdaterTest.java | 173 ++++- ...oordinatorBasicAuthorizerResourceTest.java | 256 ++++++- .../NoopBasicAuthorizerCacheNotifier.java | 8 +- .../s3/TestAWSCredentialsProvider.java | 3 + 66 files changed, 6650 insertions(+), 358 deletions(-) create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecurityAuthenticationException.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecuritySSLSocketFactory.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicAuthenticatorUserPrincipal.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthConfig.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorConfigBundle.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/CredentialsValidator.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/DBCredentialsValidator.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMapping.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingFull.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingMapBundle.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/GroupMappingAndRoleMap.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheManager.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheNotifier.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/MetadataStoragePollingBasicEscalatorCacheManager.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/BasicEscalatorMetadataStorageUpdater.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResource.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResourceHandler.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java create mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredentialBundle.java create mode 100644 extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java diff --git a/docs/content/development/extensions-core/druid-basic-security.md b/docs/content/development/extensions-core/druid-basic-security.md index 890b2738d3a7..925b7a55245d 100644 --- a/docs/content/development/extensions-core/druid-basic-security.md +++ b/docs/content/development/extensions-core/druid-basic-security.md @@ -25,8 +25,8 @@ title: "Basic Security" # Druid Basic Security This extension adds: -- an Authenticator which supports [HTTP Basic authentication](https://en.wikipedia.org/wiki/Basic_access_authentication) -- an Authorizer which implements basic role-based access control +- an Authenticator which supports [HTTP Basic authentication](https://en.wikipedia.org/wiki/Basic_access_authentication) using database or LDAP as its credentials store +- an Authorizer which implements basic role-based access control for database and LDAP users and groups Make sure to [include](../../operations/including-extensions.html) `druid-basic-security` as an extension. @@ -34,7 +34,7 @@ Please see [Authentication and Authorization](../../design/auth.html) for more i ## Configuration -The examples in the section will use "MyBasicAuthenticator" and "MyBasicAuthorizer" as names for the Authenticator and Authorizer. +The examples in the section will use "MyBasicDBAuthenticator", "MyBasicLDAPAuthenticator" and "MyBasicAuthorizer" as names for the Authenticators and Authorizer. These properties are not tied to specific Authenticator or Authorizer instances. @@ -43,23 +43,27 @@ These configuration properties should be added to the common runtime properties ### Properties |Property|Description|Default|required| |--------|-----------|-------|--------| -|`druid.auth.basic.common.pollingPeriod`|Defines in milliseconds how often nodes should poll the Coordinator for the current authenticator/authorizer database state.|60000|No| +|`druid.auth.basic.common.pollingPeriod`|Defines in milliseconds how often nodes should poll the Coordinator for the current escalator/authenticator/authorizer database state.|60000|No| |`druid.auth.basic.common.maxRandomDelay`|Defines in milliseconds the amount of random delay to add to the pollingPeriod, to spread polling requests across time.|6000|No| |`druid.auth.basic.common.maxSyncRetries`|Determines how many times a service will retry if the authentication/authorization database state sync with the Coordinator fails.|10|No| |`druid.auth.basic.common.cacheDirectory`|If defined, snapshots of the basic Authenticator and Authorizer database caches will be stored on disk in this directory. If this property is defined, when a service is starting, it will attempt to initialize its caches from these on-disk snapshots, if the service is unable to initialize its state by communicating with the Coordinator.|null|No| -### Creating an Authenticator +### Creating an Authenticator that uses database to lookup and validate credentials ``` -druid.auth.authenticatorChain=["MyBasicAuthenticator"] - -druid.auth.authenticator.MyBasicAuthenticator.type=basic -druid.auth.authenticator.MyBasicAuthenticator.initialAdminPassword=password1 -druid.auth.authenticator.MyBasicAuthenticator.initialInternalClientPassword=password2 -druid.auth.authenticator.MyBasicAuthenticator.authorizerName=MyBasicAuthorizer +druid.auth.authenticatorChain=["MyBasicDBAuthenticator"] + +druid.auth.authenticator.MyBasicDBAuthenticator.type=basic +druid.auth.authenticator.MyBasicDBAuthenticator.initialAdminPassword=password1 +druid.auth.authenticator.MyBasicDBAuthenticator.initialInternalClientPassword=password2 +druid.auth.authenticator.MyBasicDBAuthenticator.credentialsValidator.type=db +druid.auth.authenticator.MyBasicDBAuthenticator.skipOnFailure=false +druid.auth.authenticator.MyBasicDBAuthenticator.authorizerName=MyBasicAuthorizer ``` To use the Basic authenticator, add an authenticator with type `basic` to the authenticatorChain. +The authenticator needs to also define a credentialsValidator with type 'db' or 'ldap'. +If credentialsValidator is not specified, type 'db' will be used as default. Configuration of the named authenticator is assigned through properties with the form: @@ -67,18 +71,42 @@ Configuration of the named authenticator is assigned through properties with the druid.auth.authenticator.. ``` -The configuration examples in the rest of this document will use "MyBasicAuthenticator" as the name of the authenticator being configured. +The configuration examples in the rest of this document will use "MyBasicDBAuthenticator" or "MyBasicLDAPAuthenticator" as the name of the authenticator being configured. -#### Properties +#### Properties for database user authentication +|Property|Description|Default|required| +|--------|-----------|-------|--------| +|`druid.auth.authenticator.MyBasicDBAuthenticator.initialAdminPassword`|Initial [Password Provider](../../operations/password-provider.html) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.|null|No| +|`druid.auth.authenticator.MyBasicDBAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.html) for the default internal system user, used for internal node communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No| +|`druid.auth.authenticator.MyBasicDBAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid nodes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No| +|`druid.auth.authenticator.MyBasicDBAuthenticator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No| +|`druid.auth.authenticator.MyBasicDBAuthenticator.credentialIterations`|Number of iterations to use for password hashing.|10000|No| +|`druid.auth.authenticator.MyBasicDBAuthenticator.credentialsValidator.type`|The type of credentials store (db) to validate requests credentials.|db|No| +|`druid.auth.authenticator.MyBasicDBAuthenticator.skipOnFailure`|If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.|false|No| +|`druid.auth.authenticator.MyBasicDBAuthenticator.authorizerName`|Authorizer that requests should be directed to|N/A|Yes| + +#### Properties for LDAP user authentication |Property|Description|Default|required| |--------|-----------|-------|--------| -|`druid.auth.authenticator.MyBasicAuthenticator.initialAdminPassword`|Initial [Password Provider](../../operations/password-provider.html) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.|null|No| -|`druid.auth.authenticator.MyBasicAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.html) for the default internal system user, used for internal node communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No| -|`druid.auth.authenticator.MyBasicAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid nodes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No| -|`druid.auth.authenticator.MyBasicAuthenticator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No| -|`druid.auth.authenticator.MyBasicAuthenticator.credentialIterations`|Number of iterations to use for password hashing.|10000|No| -|`druid.auth.authenticator.MyBasicAuthenticator.authorizerName`|Authorizer that requests should be directed to|N/A|Yes| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialAdminPassword`|Initial [Password Provider](../../operations/password-provider.html) for the automatically created default admin user. If no password is specified, the default admin user will not be created. If the default admin user already exists, setting this property will not affect its password.|null|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.initialInternalClientPassword`|Initial [Password Provider](../../operations/password-provider.html) for the default internal system user, used for internal node communication. If no password is specified, the default internal system user will not be created. If the default internal system user already exists, setting this property will not affect its password.|null|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.enableCacheNotifications`|If true, the Coordinator will notify Druid nodes whenever a configuration change to this Authenticator occurs, allowing them to immediately update their state without waiting for polling.|true|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialIterations`|Number of iterations to use for password hashing.|10000|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.type`|The type of credentials store (ldap) to validate requests credentials.|db|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.url`|URL of the LDAP server.|null|Yes| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindUser`|LDAP bind user username.|null|Yes| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.bindPassword`|[Password Provider](../../operations/password-provider.html) LDAP bind user password.|null|Yes| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.baseDn`|The point from where the LDAP server will search for users.|null|Yes| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userSearch`|The filter/expression to use for the search. For example, (&(sAMAccountName=%s)(objectClass=user))|null|Yes| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.userAttribute`|The attribute id identifying the attribute that will be returned as part of the search. For example, sAMAccountName. |null|Yes| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.groupFilters`|Array of filters used to filter out the allowed set of groups returned from search. Filters can be begin with *, or end with ,* to provide configurational flexibility.|null|Yes| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialVerifyDuration`|The duration in seconds for how long valid credentials are verifiable within the cache when not requested.|600|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialMaxDuration`|The max duration in seconds for valid credentials that can reside in cache regardless of how often they are requested.|3600|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.credentialsValidator.credentialCacheSize`|The valid credentials cache size. The cache uses a LRU policy.|100|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.skipOnFailure`|If true and the request credential doesn't exists or isn't fully configured in the credentials store, the request will proceed to next Authenticator in the chain.|false|No| +|`druid.auth.authenticator.MyBasicLDAPAuthenticator.authorizerName`|Authorizer that requests should be directed to.|N/A|Yes| ### Creating an Escalator @@ -96,6 +124,9 @@ druid.escalator.authorizerName=MyBasicAuthorizer |`druid.escalator.internalClientUsername`|The escalator will use this username for requests made as the internal systerm user.|n/a|Yes| |`druid.escalator.internalClientPassword`|The escalator will use this [Password Provider](../../operations/password-provider.html) for requests made as the internal system user.|n/a|Yes| |`druid.escalator.authorizerName`|Authorizer that requests should be directed to.|n/a|Yes| +|`druid.escalator.enableCacheNotifications`|If true, the Coordinator will notify Druid nodes whenever a configuration change to this Escalator occurs, allowing them to immediately update their state without waiting for polling.|true|No| +|`druid.escalator.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No| +|`druid.escalator.internalClientCredentialPoll`|Defines in seconds how often Escalator will poll for its current internal client credentials|10|No| ### Creating an Authorizer @@ -105,9 +136,9 @@ druid.auth.authorizers=["MyBasicAuthorizer"] druid.auth.authorizer.MyBasicAuthorizer.type=basic ``` -To use the Basic authorizer, add an authenticator with type `basic` to the authorizers list. +To use the Basic authorizer, add an authorizer with type `basic` to the authorizers list. -Configuration of the named authenticator is assigned through properties with the form: +Configuration of the named authorizer is assigned through properties with the form: ``` druid.auth.authorizer.. @@ -118,6 +149,9 @@ druid.auth.authorizer.. |--------|-----------|-------|--------| |`druid.auth.authorizer.MyBasicAuthorizer.enableCacheNotifications`|If true, the Coordinator will notify Druid nodes whenever a configuration change to this Authorizer occurs, allowing them to immediately update their state without waiting for polling.|true|No| |`druid.auth.authorizer.MyBasicAuthorizer.cacheNotificationTimeout`|The timeout in milliseconds for the cache notifications.|5000|No| +|`druid.auth.authorizer.MyBasicAuthorizer.initialAdminUser`|The initial admin user with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.|admin|No| +|`druid.auth.authorizer.MyBasicAuthorizer.initialAdminRole`|The initial admin role to create if it doesn't already exists.|admin|No| +|`druid.auth.authorizer.MyBasicAuthorizer.initialAdminGroupMapping`|The initial admin group mapping with role defined in initialAdminRole property if specified, otherwise the default admin role will be assigned.|admin|No| ## Usage @@ -146,15 +180,34 @@ Delete the user with name {userName} `POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/users/{userName}/credentials)` Assign a password used for HTTP basic authentication for {userName} Content: JSON password request object - Example request body: - ``` { "password": "helloworld" } ``` +`GET(/druid-ext/basic-security/authentication/db/{authenticatorName}/config)` +Return the authenticator current set of updatable configuration. + +`POST(/druid-ext/basic-security/authentication/db/{authenticatorName}/config)` +Update the authenticator current set of updatable configuration. +Content: JSON config request object +Example request body: +``` +{ + "url": "ldaps://host:port", + "bindUser": "DHC\\username", + "bindPassword": "password", + "baseDn": "DC=corp,DC=company,DC=com", + "userSearch": "(&(sAMAccountName=%s)(objectClass=user))", + "userAttribute": "sAMAccountName", + "groupFilters": [ + "*,OU=Groupings,DC=corp,DC=company,DC=com" + ] +} +``` + ##### Cache Load Status `GET(/druid-ext/basic-security/authentication/loadStatus)` Return the current load status of the local caches of the authentication database. @@ -178,6 +231,30 @@ Create a new user with name {userName} `DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName})` Delete the user with name {userName} +##### Group mapping Creation/Deletion +`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupmappings)` +Return a list of all group mappings. + +`GET(/druid-ext/basic-security/authorization/db/{authorizerName}/groupmappings/{groupMappingName})` +Return the group mapping and role information of the group mapping with name {groupMappingName} + +`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupmappings/{groupMappingName})` +Create a new group mapping with name {groupMappingName} +Content: JSON group mapping object +Example request body: + +``` +{ + "name": "user", + "groupPattern": "CN=aaa,OU=aaa,OU=Groupings,DC=corp,DC=company,DC=com", + "roles": [ + "user" + ] +} +``` + +`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupmappings/{groupMappingName})` +Delete the group mapping with name {groupMappingName} #### Role Creation/Deletion `GET(/druid-ext/basic-security/authorization/db/{authorizerName}/roles)` @@ -201,6 +278,12 @@ Assign role {roleName} to user {userName}. `DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/users/{userName}/roles/{roleName})` Unassign role {roleName} from user {userName} +`POST(/druid-ext/basic-security/authorization/db/{authorizerName}/groupmappings/{groupMappingName}/roles/{roleName})` +Assign role {roleName} to group mapping {groupMappingName}. + +`DELETE(/druid-ext/basic-security/authorization/db/{authorizerName}/groupmappings/{groupMappingName}/roles/{roleName})` +Unassign role {roleName} from group mapping {groupMappingName} + #### Permissions `POST(/druid-ext/basic-security/authorization/db/{authorizerName}/roles/{roleName}/permissions)` diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthDBConfig.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthDBConfig.java index 7e084cb63beb..3e6eb250b45e 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthDBConfig.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthDBConfig.java @@ -27,23 +27,62 @@ public class BasicAuthDBConfig private final PasswordProvider initialAdminPassword; private final PasswordProvider initialInternalClientPassword; + private final String initialAdminUser; + private final String initialAdminRole; + private final String initialAdminGroupMapping; private final boolean enableCacheNotifications; private final long cacheNotificationTimeout; private final int iterations; + private final String url; + private final String bindUser; + private final PasswordProvider bindPassword; + private final String baseDn; + private final String userSearch; + private final String userAttribute; + private final String[] groupFilters; + private final Integer credentialVerifyDuration; + private final Integer credentialMaxDuration; + private final Integer credentialCacheSize; public BasicAuthDBConfig( final PasswordProvider initialAdminPassword, final PasswordProvider initialInternalClientPassword, - final Boolean enableCacheNotifications, - final Long cacheNotificationTimeout, - final int iterations + final String initialAdminUser, + final String initialAdminRole, + final String initialAdminGroupMapping, + final boolean enableCacheNotifications, + final long cacheNotificationTimeout, + final int credentialIterations, + final String url, + final String bindUser, + final PasswordProvider bindPassword, + final String baseDn, + final String userSearch, + final String userAttribute, + final String[] groupFilters, + final Integer credentialVerifyDuration, + final Integer credentialMaxDuration, + final Integer credentialCacheSize ) { this.initialAdminPassword = initialAdminPassword; this.initialInternalClientPassword = initialInternalClientPassword; + this.initialAdminUser = initialAdminUser; + this.initialAdminRole = initialAdminRole; + this.initialAdminGroupMapping = initialAdminGroupMapping; this.enableCacheNotifications = enableCacheNotifications; this.cacheNotificationTimeout = cacheNotificationTimeout; - this.iterations = iterations; + this.iterations = credentialIterations; + this.url = url; + this.bindUser = bindUser; + this.bindPassword = bindPassword; + this.baseDn = baseDn; + this.userSearch = userSearch; + this.userAttribute = userAttribute; + this.groupFilters = groupFilters; + this.credentialVerifyDuration = credentialVerifyDuration; + this.credentialMaxDuration = credentialMaxDuration; + this.credentialCacheSize = credentialCacheSize; } public PasswordProvider getInitialAdminPassword() @@ -56,6 +95,21 @@ public PasswordProvider getInitialInternalClientPassword() return initialInternalClientPassword; } + public String getInitialAdminUser() + { + return initialAdminUser; + } + + public String getInitialAdminRole() + { + return initialAdminRole; + } + + public String getInitialAdminGroupMapping() + { + return initialAdminGroupMapping; + } + public boolean isEnableCacheNotifications() { return enableCacheNotifications; @@ -70,4 +124,54 @@ public int getIterations() { return iterations; } + + public String getUrl() + { + return url; + } + + public String getBindUser() + { + return bindUser; + } + + public PasswordProvider getBindPassword() + { + return bindPassword; + } + + public String getBaseDn() + { + return baseDn; + } + + public String getUserSearch() + { + return userSearch; + } + + public String getUserAttribute() + { + return userAttribute; + } + + public String[] getGroupFilters() + { + return groupFilters; + } + + public Integer getCredentialVerifyDuration() + { + return credentialVerifyDuration; + } + + public Integer getCredentialMaxDuration() + { + return credentialMaxDuration; + } + + public Integer getCredentialCacheSize() + { + return credentialCacheSize; + } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthUtils.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthUtils.java index d66dc1ddbac1..820cd7074df3 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthUtils.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthUtils.java @@ -25,10 +25,14 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; +import org.apache.druid.security.basic.authorization.entity.GroupMappingAndRoleMap; import org.apache.druid.security.basic.authorization.entity.UserAndRoleMap; +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; import javax.annotation.Nullable; import javax.crypto.SecretKey; @@ -48,7 +52,9 @@ public class BasicAuthUtils private static final Logger log = new Logger(BasicAuthUtils.class); private static final SecureRandom SECURE_RANDOM = new SecureRandom(); public static final String ADMIN_NAME = "admin"; + public static final String ADMIN_GROUP_MAPPING_NAME = "adminGroupMapping"; public static final String INTERNAL_USER_NAME = "druid_system"; + public static final String GROUPS_CONTEXT_KEY = "groups"; // PBKDF2WithHmacSHA512 is chosen since it has built-in support in Java8. // Argon2 (https://github.com/p-h-c/phc-winner-argon2) is newer but the only presently @@ -57,6 +63,9 @@ public class BasicAuthUtils // 256-bit salt should be more than sufficient for uniqueness, expected user count is on the order of thousands. public static final int SALT_LENGTH = 32; public static final int DEFAULT_KEY_ITERATIONS = 10000; + public static final int DEFAULT_CREDENTIAL_VERIFY_DURATION_SECONDS = 600; + public static final int DEFAULT_CREDENTIAL_MAX_DURATION_SECONDS = 3600; + public static final int DEFAULT_CREDENTIAL_CACHE_SIZE = 100; public static final int KEY_LENGTH = 512; public static final String ALGORITHM = "PBKDF2WithHmacSHA512"; @@ -70,6 +79,11 @@ public class BasicAuthUtils { }; + public static final TypeReference AUTHORIZER_GROUP_MAPPING_MAP_TYPE_REFERENCE = + new TypeReference>() + { + }; + public static final TypeReference AUTHORIZER_ROLE_MAP_TYPE_REFERENCE = new TypeReference>() { @@ -80,6 +94,11 @@ public class BasicAuthUtils { }; + public static final TypeReference AUTHORIZER_GROUP_MAPPING_AND_ROLE_MAP_TYPE_REFERENCE = + new TypeReference() + { + }; + public static byte[] hashPassword(final char[] password, final byte[] salt, final int iterations) { try { @@ -155,7 +174,7 @@ public static Map deserializeAuthenticatorUserMa userMap = objectMapper.readValue(userMapBytes, AUTHENTICATOR_USER_MAP_TYPE_REFERENCE); } catch (IOException ioe) { - throw new RuntimeException(ioe); + throw new RuntimeException("Couldn't deserialize authenticator userMap!", ioe); } } return userMap; @@ -170,7 +189,7 @@ public static byte[] serializeAuthenticatorUserMap( return objectMapper.writeValueAsBytes(userMap); } catch (IOException ioe) { - throw new ISE(ioe, "WTF? Couldn't serialize userMap!"); + throw new ISE(ioe, "Couldn't serialize authenticator userMap!"); } } @@ -187,7 +206,7 @@ public static Map deserializeAuthorizerUserMap( userMap = objectMapper.readValue(userMapBytes, BasicAuthUtils.AUTHORIZER_USER_MAP_TYPE_REFERENCE); } catch (IOException ioe) { - throw new RuntimeException(ioe); + throw new RuntimeException("Couldn't deserialize authorizer userMap!", ioe); } } return userMap; @@ -199,7 +218,36 @@ public static byte[] serializeAuthorizerUserMap(ObjectMapper objectMapper, Map deserializeAuthorizerGroupMappingMap( + ObjectMapper objectMapper, + byte[] groupMappingMapBytes + ) + { + Map groupMappingMap; + if (groupMappingMapBytes == null) { + groupMappingMap = new HashMap<>(); + } else { + try { + groupMappingMap = objectMapper.readValue(groupMappingMapBytes, BasicAuthUtils.AUTHORIZER_GROUP_MAPPING_MAP_TYPE_REFERENCE); + } + catch (IOException ioe) { + throw new RuntimeException("Couldn't deserialize authorizer groupMappingMap!", ioe); + } + } + return groupMappingMap; + } + + public static byte[] serializeAuthorizerGroupMappingMap(ObjectMapper objectMapper, Map groupMappingMap) + { + try { + return objectMapper.writeValueAsBytes(groupMappingMap); + } + catch (IOException ioe) { + throw new ISE(ioe, "Couldn't serialize authorizer groupMappingMap!"); } } @@ -216,7 +264,7 @@ public static Map deserializeAuthorizerRoleMap( roleMap = objectMapper.readValue(roleMapBytes, BasicAuthUtils.AUTHORIZER_ROLE_MAP_TYPE_REFERENCE); } catch (IOException ioe) { - throw new RuntimeException(ioe); + throw new RuntimeException("Couldn't deserialize authorizer roleMap!", ioe); } } return roleMap; @@ -228,7 +276,69 @@ public static byte[] serializeAuthorizerRoleMap(ObjectMapper objectMapper, Map getJacksonModules() { @@ -163,7 +221,8 @@ public List getJacksonModules() new SimpleModule("BasicDruidSecurity").registerSubtypes( BasicHTTPAuthenticator.class, BasicHTTPEscalator.class, - BasicRoleBasedAuthorizer.class + BasicRoleBasedAuthorizer.class, + NettyHttpClient.class ) ); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecuritySSLSocketFactory.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecuritySSLSocketFactory.java new file mode 100644 index 000000000000..ff860796df54 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecuritySSLSocketFactory.java @@ -0,0 +1,136 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic; + +import org.apache.druid.java.util.common.logger.Logger; + +import javax.net.SocketFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.SSLSocketFactory; +import javax.net.ssl.TrustManager; +import javax.net.ssl.X509TrustManager; +import java.io.IOException; +import java.net.InetAddress; +import java.net.Socket; +import java.security.KeyManagementException; +import java.security.NoSuchAlgorithmException; +import java.security.SecureRandom; +import java.security.cert.X509Certificate; + +public class BasicSecuritySSLSocketFactory extends SSLSocketFactory +{ + private static final Logger LOG = new Logger(BasicSecuritySSLSocketFactory.class); + private SSLSocketFactory sf; + + public BasicSecuritySSLSocketFactory() + { + try { + SSLContext ctx = SSLContext.getInstance("TLS"); + ctx.init( + null, + new TrustManager[] {new BasicSecurityTrustManager()}, + new SecureRandom()); + sf = ctx.getSocketFactory(); + } + catch (NoSuchAlgorithmException | KeyManagementException e) { + throw new IllegalArgumentException( + "Unable to initialize socket factory", e); + } + } + + public static SocketFactory getDefault() + { + return new BasicSecuritySSLSocketFactory(); + } + + @Override + public String[] getDefaultCipherSuites() + { + return sf.getDefaultCipherSuites(); + } + + @Override + public String[] getSupportedCipherSuites() + { + return sf.getSupportedCipherSuites(); + } + + @Override + public Socket createSocket( + Socket s, + String host, + int port, + boolean autoClose) throws IOException + { + return sf.createSocket(s, host, port, autoClose); + } + + @Override + public Socket createSocket(String host, int port) throws IOException + { + return sf.createSocket(host, port); + } + + @Override + public Socket createSocket( + String host, + int port, + InetAddress localHost, + int localPort) throws IOException + { + return sf.createSocket(host, port, localHost, localPort); + } + + @Override + public Socket createSocket(InetAddress host, int port) throws IOException + { + return sf.createSocket(host, port); + } + + @Override + public Socket createSocket( + InetAddress address, + int port, + InetAddress localAddress, + int localPort) throws IOException + { + return sf.createSocket(address, port, localAddress, localPort); + } + + static class BasicSecurityTrustManager implements X509TrustManager + { + @Override + public void checkClientTrusted(X509Certificate[] chain, String authType) + { + } + + @Override + public void checkServerTrusted(X509Certificate[] chain, String authType) + { + } + + @Override + public X509Certificate[] getAcceptedIssuers() + { + return new X509Certificate[0]; + } + } + +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java index f1e61ebfba75..f181f24d3cfa 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java @@ -75,6 +75,7 @@ public class CommonCacheNotifier private final HttpClient httpClient; private final BlockingQueue> updateQueue; private final Map itemConfigMap; + private final BasicAuthDBConfig itemConfig; private final String baseUrl; private final String callerName; private final ExecutorService exec; @@ -94,6 +95,25 @@ public CommonCacheNotifier( this.discoveryProvider = discoveryProvider; this.httpClient = httpClient; this.baseUrl = baseUrl; + this.itemConfig = null; + } + + public CommonCacheNotifier( + BasicAuthDBConfig itemConfig, + DruidNodeDiscoveryProvider discoveryProvider, + HttpClient httpClient, + String baseUrl, + String callerName + ) + { + this.itemConfig = itemConfig; + this.exec = Execs.scheduledSingleThreaded(StringUtils.format("%s-notifierThread-", callerName) + "%d"); + this.callerName = callerName; + this.updateQueue = new LinkedBlockingQueue<>(); + this.discoveryProvider = discoveryProvider; + this.httpClient = httpClient; + this.baseUrl = baseUrl; + this.itemConfigMap = null; } public void start() @@ -106,23 +126,26 @@ public void start() Pair update = updateQueue.take(); String authorizer = update.lhs; byte[] serializedMap = update.rhs; - BasicAuthDBConfig authorizerConfig = itemConfigMap.get(update.lhs); - if (!authorizerConfig.isEnableCacheNotifications()) { + + BasicAuthDBConfig dbConfig = itemConfigMap == null ? itemConfig : itemConfigMap.get(update.lhs); + if (!dbConfig.isEnableCacheNotifications()) { continue; } LOG.debug(callerName + ":Sending cache update notifications"); // Best effort, if a notification fails, the remote node will eventually poll to update its state // We wait for responses however, to avoid flooding remote nodes with notifications. - List> futures = sendUpdate( - authorizer, - serializedMap - ); + List> futures; + if (authorizer != null) { + futures = sendUpdate(authorizer, serializedMap); + } else { + futures = sendUpdate(serializedMap); + } try { List responses = Futures.allAsList(futures) .get( - authorizerConfig.getCacheNotificationTimeout(), + dbConfig.getCacheNotificationTimeout(), TimeUnit.MILLISECONDS ); @@ -156,20 +179,28 @@ public void addUpdate(String updatedItemName, byte[] updatedItemData) ); } - private List> sendUpdate(String updatedAuthorizerPrefix, byte[] serializedUserMap) + public void addUpdate(byte[] updatedItemData) + { + updateQueue.add(new Pair<>(null, updatedItemData)); + } + + private List> sendUpdate(String updatedAuthenticatorPrefix, byte[] serializedEntity) { List> futures = new ArrayList<>(); for (NodeType nodeType : NODE_TYPES) { DruidNodeDiscovery nodeDiscovery = discoveryProvider.getForNodeType(nodeType); Collection nodes = nodeDiscovery.getAllNodes(); for (DiscoveryDruidNode node : nodes) { - URL listenerURL = getListenerURL(node.getDruidNode(), baseUrl, updatedAuthorizerPrefix); + URL listenerURL = getListenerURL( + node.getDruidNode(), + StringUtils.format(baseUrl, StringUtils.urlEncode(updatedAuthenticatorPrefix)) + ); // best effort, if this fails, remote node will poll and pick up the update eventually Request req = new Request(HttpMethod.POST, listenerURL); - req.setContent(MediaType.APPLICATION_JSON, serializedUserMap); + req.setContent(MediaType.APPLICATION_JSON, serializedEntity); - BasicAuthDBConfig itemConfig = itemConfigMap.get(updatedAuthorizerPrefix); + BasicAuthDBConfig itemConfig = itemConfigMap.get(updatedAuthenticatorPrefix); ListenableFuture future = httpClient.go( req, @@ -182,18 +213,43 @@ private List> sendUpdate(String updatedAu return futures; } - private URL getListenerURL(DruidNode druidNode, String baseUrl, String itemName) + private List> sendUpdate(byte[] serializedEntity) + { + List> futures = new ArrayList<>(); + for (NodeType nodeType : NODE_TYPES) { + DruidNodeDiscovery nodeDiscovery = discoveryProvider.getForNodeType(nodeType); + Collection nodes = nodeDiscovery.getAllNodes(); + for (DiscoveryDruidNode node : nodes) { + URL listenerURL = getListenerURL(node.getDruidNode(), baseUrl); + + // best effort, if this fails, remote node will poll and pick up the update eventually + Request req = new Request(HttpMethod.POST, listenerURL); + req.setContent(MediaType.APPLICATION_JSON, serializedEntity); + + ListenableFuture future = httpClient.go( + req, + new ResponseHandler(), + Duration.millis(itemConfig.getCacheNotificationTimeout()) + ); + futures.add(future); + } + } + return futures; + } + + private URL getListenerURL(DruidNode druidNode, String baseUrl) { try { return new URL( druidNode.getServiceScheme(), druidNode.getHost(), druidNode.getPortToUse(), - StringUtils.format(baseUrl, StringUtils.urlEncode(itemName)) + baseUrl ); } catch (MalformedURLException mue) { - LOG.error(callerName + ":WTF? Malformed url for DruidNode[%s] and itemName[%s]", druidNode, itemName); + LOG.error(callerName + ":WTF? Malformed url for DruidNode[%s] and baseUrl[%s]", druidNode, baseUrl); + throw new RuntimeException(mue); } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicAuthenticatorUserPrincipal.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicAuthenticatorUserPrincipal.java new file mode 100644 index 000000000000..810c32a47cf9 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicAuthenticatorUserPrincipal.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authentication; + +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.security.basic.BasicAuthUtils; +import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; + +import javax.naming.ldap.LdapName; +import java.security.Principal; +import java.time.Instant; +import java.util.Arrays; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +public class BasicAuthenticatorUserPrincipal implements Principal +{ + private static final Logger LOG = new Logger(BasicAuthenticatorUserPrincipal.class); + + private final String name; + private final BasicAuthenticatorCredentials credentials; + private final Set groups; + private final Instant createdAt; + private final AtomicReference lastVerified = new AtomicReference<>(); + + public BasicAuthenticatorUserPrincipal( + String name, + BasicAuthenticatorCredentials credentials, + Set groups + ) + { + this(name, credentials, groups, Instant.now()); + } + + private BasicAuthenticatorUserPrincipal( + String name, + BasicAuthenticatorCredentials credentials, + Set groups, + Instant createdAt + ) + { + Objects.requireNonNull(name, "name is required"); + Objects.requireNonNull(credentials, "credentials is required"); + Objects.requireNonNull(groups, "groups is required"); + Objects.requireNonNull(createdAt, "createdAt is required"); + + this.name = name; + this.credentials = credentials; + this.groups = groups; + this.createdAt = createdAt; + this.lastVerified.set(createdAt); + } + + @Override + public String getName() + { + return this.name; + } + + public Set getGroups() + { + return groups; + } + + public Instant getCreatedAt() + { + return createdAt; + } + + public Instant getLastVerified() + { + return lastVerified.get(); + } + + public boolean hasSameCredentials(char[] password) + { + byte[] recalculatedHash = BasicAuthUtils.hashPassword( + password, + this.credentials.getSalt(), + this.credentials.getIterations() + ); + if (Arrays.equals(recalculatedHash, credentials.getHash())) { + this.lastVerified.set(Instant.now()); + LOG.debug("Refereshing lastVerified principal user '%s'", this.name); + return true; + } else { + return false; + } + } + + public boolean isExpired(int duration, int maxDuration) + { + long now = System.currentTimeMillis(); + long cutoff = now - (duration * 1000L); + if (this.lastVerified.get().toEpochMilli() < cutoff) { + return true; + } else { + long maxCutoff = now - (maxDuration * 1000L); + if (this.createdAt.toEpochMilli() < maxCutoff) { + return true; + } else { + return false; + } + } + } + + @Override + public String toString() + { + return StringUtils.format( + "BasicAuthenticatorUserPrincipal[name=%s, groups=%s, createdAt=%s, lastVerified=%s]", + name, + groups.stream().map(LdapName::toString).collect(Collectors.joining(",", "{", "}")), + createdAt, + lastVerified); + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java index 98d4ecf9ef63..5bc50dbabb32 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java @@ -24,13 +24,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.google.inject.Provider; -import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.PasswordProvider; import org.apache.druid.security.basic.BasicAuthDBConfig; import org.apache.druid.security.basic.BasicAuthUtils; +import org.apache.druid.security.basic.BasicSecurityAuthenticationException; import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; -import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; -import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; +import org.apache.druid.security.basic.authentication.validator.CredentialsValidator; +import org.apache.druid.security.basic.authentication.validator.DBCredentialsValidator; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.Authenticator; @@ -46,17 +47,20 @@ import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletResponse; import java.io.IOException; -import java.util.Arrays; import java.util.EnumSet; import java.util.Map; @JsonTypeName("basic") public class BasicHTTPAuthenticator implements Authenticator { + private static final Logger LOG = new Logger(BasicHTTPAuthenticator.class); + private final Provider cacheManager; private final String name; private final String authorizerName; private final BasicAuthDBConfig dbConfig; + private final CredentialsValidator credentialsValidator; + private final boolean skipOnFailure; @JsonCreator public BasicHTTPAuthenticator( @@ -67,7 +71,19 @@ public BasicHTTPAuthenticator( @JsonProperty("initialInternalClientPassword") PasswordProvider initialInternalClientPassword, @JsonProperty("enableCacheNotifications") Boolean enableCacheNotifications, @JsonProperty("cacheNotificationTimeout") Long cacheNotificationTimeout, - @JsonProperty("credentialIterations") Integer credentialIterations + @JsonProperty("credentialIterations") Integer credentialIterations, + @JsonProperty("skipOnFailure") Boolean skipOnFailure, + @JsonProperty("credentialsValidator") CredentialsValidator credentialsValidator, + @JsonProperty("credentialsValidator.url") String url, + @JsonProperty("credentialsValidator.bindUser") String bindUser, + @JsonProperty("credentialsValidator.bindPassword") PasswordProvider bindPassword, + @JsonProperty("credentialsValidator.baseDn") String baseDn, + @JsonProperty("credentialsValidator.userSearch") String userSearch, + @JsonProperty("credentialsValidator.userAttribute") String userAttribute, + @JsonProperty("credentialsValidator.groupFilters") String[] groupFilters, + @JsonProperty("credentialsValidator.credentialVerifyDuration") Integer credentialVerifyDuration, + @JsonProperty("credentialsValidator.credentialMaxDuration") Integer credentialMaxDuration, + @JsonProperty("credentialsValidator.credentialCacheSize") Integer credentialCacheSize ) { this.name = name; @@ -75,11 +91,30 @@ public BasicHTTPAuthenticator( this.dbConfig = new BasicAuthDBConfig( initialAdminPassword, initialInternalClientPassword, + null, + null, + null, enableCacheNotifications == null ? true : enableCacheNotifications, cacheNotificationTimeout == null ? BasicAuthDBConfig.DEFAULT_CACHE_NOTIFY_TIMEOUT_MS : cacheNotificationTimeout, - credentialIterations == null ? BasicAuthUtils.DEFAULT_KEY_ITERATIONS : credentialIterations + credentialIterations == null ? BasicAuthUtils.DEFAULT_KEY_ITERATIONS : credentialIterations, + url, + bindUser, + bindPassword, + baseDn, + userSearch, + userAttribute, + groupFilters, + credentialVerifyDuration, + credentialMaxDuration, + credentialCacheSize ); this.cacheManager = cacheManager; + if (credentialsValidator == null) { + this.credentialsValidator = new DBCredentialsValidator(this.cacheManager); + } else { + this.credentialsValidator = credentialsValidator; + } + this.skipOnFailure = skipOnFailure; } @Override @@ -105,11 +140,7 @@ public AuthenticationResult authenticateJDBCContext(Map context) return null; } - if (checkCredentials(user, password.toCharArray())) { - return new AuthenticationResult(user, authorizerName, name, null); - } else { - return null; - } + return credentialsValidator.validateCredentials(name, authorizerName, user, password.toCharArray()); } @@ -165,8 +196,7 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo } // At this point, encodedUserSecret is not null, indicating that the request intends to perform - // Basic HTTP authentication. If any errors occur with the authentication, we send a 401 response immediately - // and do not proceed further down the filter chain. + // Basic HTTP authentication. String decodedUserSecret = BasicAuthUtils.decodeUserSecret(encodedUserSecret); if (decodedUserSecret == null) { // We recognized a Basic auth header, but could not decode the user secret. @@ -176,6 +206,7 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo String[] splits = decodedUserSecret.split(":"); if (splits.length != 2) { + // The decoded user secret is not of the right format httpResp.sendError(HttpServletResponse.SC_UNAUTHORIZED); return; } @@ -183,13 +214,33 @@ public void doFilter(ServletRequest servletRequest, ServletResponse servletRespo String user = splits[0]; char[] password = splits[1].toCharArray(); - if (checkCredentials(user, password)) { - AuthenticationResult authenticationResult = new AuthenticationResult(user, authorizerName, name, null); - servletRequest.setAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult); - filterChain.doFilter(servletRequest, servletResponse); - } else { - httpResp.sendError(HttpServletResponse.SC_UNAUTHORIZED); + // If any authentication error occurs we send a 401 response immediately and do not proceed further down the filter chain. + // If the authentication result is null and skipOnFailure property is false, we send a 401 response and do not proceed + // further down the filter chain. If the authentication result is null and skipOnFailure is true then move on to the next filter. + // Authentication results, for instance, can be null if a user doesn't exists within a user store + try { + AuthenticationResult authenticationResult = credentialsValidator.validateCredentials( + name, + authorizerName, + user, + password + ); + if (authenticationResult != null) { + servletRequest.setAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT, authenticationResult); + filterChain.doFilter(servletRequest, servletResponse); + } else { + if (skipOnFailure) { + LOG.info("Skipping failed authenticator %s ", name); + filterChain.doFilter(servletRequest, servletResponse); + } else { + httpResp.sendError(HttpServletResponse.SC_UNAUTHORIZED); + } + } } + catch (BasicSecurityAuthenticationException ex) { + httpResp.sendError(HttpServletResponse.SC_UNAUTHORIZED, ex.getMessage()); + } + } @Override @@ -198,29 +249,4 @@ public void destroy() } } - - private boolean checkCredentials(String username, char[] password) - { - Map userMap = cacheManager.get().getUserMap(name); - if (userMap == null) { - throw new IAE("No authenticator found with prefix: [%s]", name); - } - - BasicAuthenticatorUser user = userMap.get(username); - if (user == null) { - return false; - } - BasicAuthenticatorCredentials credentials = user.getCredentials(); - if (credentials == null) { - return false; - } - - byte[] recalculatedHash = BasicAuthUtils.hashPassword( - password, - credentials.getSalt(), - credentials.getIterations() - ); - - return Arrays.equals(recalculatedHash, credentials.getHash()); - } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPEscalator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPEscalator.java index dc84ab162f0c..4caa3a4b72e3 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPEscalator.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPEscalator.java @@ -19,49 +19,143 @@ package org.apache.druid.security.basic.authentication; +import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.inject.Provider; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.CredentialedHttpClient; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.auth.BasicCredentials; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.auth.Credentials; import org.apache.druid.metadata.PasswordProvider; +import org.apache.druid.security.basic.BasicAuthDBConfig; +import org.apache.druid.security.basic.escalator.db.cache.BasicEscalatorCacheManager; +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.Escalator; +import java.util.concurrent.atomic.AtomicReference; + @JsonTypeName("basic") public class BasicHTTPEscalator implements Escalator { - private final String internalClientUsername; - private final PasswordProvider internalClientPassword; + private static final Logger LOG = new Logger(BasicHTTPEscalator.class); + public static final int DEFAULT_INTERNAL_CLIENT_CREDENTIAL_POLL_SECONDS = 10; + + private final Provider cacheManager; private final String authorizerName; + private final BasicAuthDBConfig dbConfig; + private final AtomicReference cachedEscalatorCredential; + private final AtomicReference lastVerified; + private final int internalClientCredentialPoll; @JsonCreator public BasicHTTPEscalator( + @JacksonInject Provider cacheManager, @JsonProperty("authorizerName") String authorizerName, @JsonProperty("internalClientUsername") String internalClientUsername, - @JsonProperty("internalClientPassword") PasswordProvider internalClientPassword + @JsonProperty("internalClientPassword") PasswordProvider internalClientPassword, + @JsonProperty("enableCacheNotifications") Boolean enableCacheNotifications, + @JsonProperty("cacheNotificationTimeout") Long cacheNotificationTimeout, + @JsonProperty("internalClientCredentialPoll") Integer internalClientCredentialPoll ) { + this.cacheManager = cacheManager; this.authorizerName = authorizerName; - this.internalClientUsername = internalClientUsername; - this.internalClientPassword = internalClientPassword; + this.cachedEscalatorCredential = new AtomicReference<>( + new BasicEscalatorCredential(internalClientUsername, internalClientPassword.getPassword()) + ); + this.dbConfig = new BasicAuthDBConfig( + null, + null, + null, + null, + null, + enableCacheNotifications == null ? true : enableCacheNotifications, + cacheNotificationTimeout == null ? BasicAuthDBConfig.DEFAULT_CACHE_NOTIFY_TIMEOUT_MS : cacheNotificationTimeout, + 0, + null, + null, + null, + null, + null, + null, + null, null, + null, + null + ); + this.internalClientCredentialPoll = internalClientCredentialPoll != null ? internalClientCredentialPoll : DEFAULT_INTERNAL_CLIENT_CREDENTIAL_POLL_SECONDS; + Long now = System.currentTimeMillis(); + lastVerified = new AtomicReference<>(now); } @Override public HttpClient createEscalatedClient(HttpClient baseClient) { - return new CredentialedHttpClient( - new BasicCredentials(internalClientUsername, internalClientPassword.getPassword()), - baseClient - ); + LOG.debug("----------- Creating escalated client"); + return new CredentialedHttpClient(new BasicEscalatorCredentials(), baseClient); } @Override public AuthenticationResult createEscalatedAuthenticationResult() { + BasicEscalatorCredential basicEscalatorCredential = getOrUpdateEscalatorCredentials(); + LOG.debug("----------- Creating escalated authentication result. username: %s", basicEscalatorCredential.getUsername()); // if you found your self asking why the authenticatedBy field is set to null please read this: // https://github.com/apache/incubator-druid/pull/5706#discussion_r185940889 - return new AuthenticationResult(internalClientUsername, authorizerName, null, null); + return new AuthenticationResult( + basicEscalatorCredential.getUsername(), + authorizerName, + null, + null); + } + + public BasicAuthDBConfig getDbConfig() + { + return dbConfig; + } + + private final class BasicEscalatorCredentials implements Credentials + { + @Override + public Request addCredentials(Request builder) + { + BasicEscalatorCredential basicEscalatorCredential = getOrUpdateEscalatorCredentials(); + LOG.debug("----------- Adding escalator credentials. username: %s", basicEscalatorCredential.getUsername()); + return builder.setBasicAuthentication( + basicEscalatorCredential.getUsername(), + basicEscalatorCredential.getPassword() + ); + } + } + + private BasicEscalatorCredential getOrUpdateEscalatorCredentials() + { + BasicEscalatorCredential escalatorCredential = cachedEscalatorCredential.get(); + BasicEscalatorCredential polledEscalatorCredential; + long now = System.currentTimeMillis(); + long cutoff = now - (internalClientCredentialPoll * 1000L); + + if (lastVerified.get() < cutoff) { + lastVerified.set(now); + try { + polledEscalatorCredential = cacheManager.get().getEscalatorCredential(); + } + catch (Exception ex) { + polledEscalatorCredential = null; + } + + if (polledEscalatorCredential == null || polledEscalatorCredential.equals(escalatorCredential)) { + LOG.debug("----------- Escalator credentials validated, no need to modify"); + return escalatorCredential; + } else { + LOG.debug("----------- Modified escalator credentials found, reloading"); + cachedEscalatorCredential.set(polledEscalatorCredential); + return polledEscalatorCredential; + } + } + return escalatorCredential; } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheManager.java index 70851058736a..1f21c67b911f 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheManager.java @@ -19,6 +19,7 @@ package org.apache.druid.security.basic.authentication.db.cache; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; import java.util.Map; @@ -30,12 +31,11 @@ public interface BasicAuthenticatorCacheManager { /** - * Update this cache manager's local state with fresh information pushed by the coordinator. - * + * Update this cache manager's local state of user map with fresh information pushed by the coordinator. * @param authenticatorPrefix The name of the authenticator this update applies to. * @param serializedUserMap The updated, serialized user map */ - void handleAuthenticatorUpdate(String authenticatorPrefix, byte[] serializedUserMap); + void handleAuthenticatorUserMapUpdate(String authenticatorPrefix, byte[] serializedUserMap); /** * Return the cache manager's local view of the user map for the authenticator named `authenticatorPrefix`. @@ -44,4 +44,20 @@ public interface BasicAuthenticatorCacheManager * @return User map */ Map getUserMap(String authenticatorPrefix); + + /** + * Update this cache manager's config local state with fresh information pushed by the coordinator. + * + * @param authenticatorPrefix The name of the authenticator this update applies to. + * @param serializedConfig The updated, serialized config + */ + void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig); + + /** + * Return the cache manager's local view of config for the authenticator named `authenticatorPrefix`. + * + * @param authenticatorPrefix The name of the authenticator + * @return Config map + */ + BasicAuthConfig getConfig(String authenticatorPrefix); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheNotifier.java index 95a631a29218..718b5149fb27 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheNotifier.java @@ -30,5 +30,13 @@ public interface BasicAuthenticatorCacheNotifier * @param updatedAuthenticatorPrefix Name of authenticator being updated * @param updatedUserMap User map state */ - void addUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserMap); + void addUserUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserMap); + + /** + * Send the config state contained in updatedConfig to all non-coordinator Druid services + * + * @param updatedAuthenticatorPrefix Name of authenticator being updated + * @param updatedConfig Config state + */ + void addConfigUpdate(String updatedAuthenticatorPrefix, byte[] updatedConfig); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java index 465b76a3e7c6..5eff84fbda5d 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java @@ -42,9 +42,9 @@ @ManageLifecycle public class CoordinatorBasicAuthenticatorCacheNotifier implements BasicAuthenticatorCacheNotifier { - private final LifecycleLock lifecycleLock = new LifecycleLock(); - private CommonCacheNotifier cacheNotifier; + private CommonCacheNotifier userCacheNotifier; + private CommonCacheNotifier configCacheNotifier; @Inject public CoordinatorBasicAuthenticatorCacheNotifier( @@ -53,24 +53,32 @@ public CoordinatorBasicAuthenticatorCacheNotifier( @EscalatedClient HttpClient httpClient ) { - cacheNotifier = new CommonCacheNotifier( + userCacheNotifier = new CommonCacheNotifier( initAuthenticatorConfigMap(authenticatorMapper), discoveryProvider, httpClient, "/druid-ext/basic-security/authentication/listen/%s", "CoordinatorBasicAuthenticatorCacheNotifier" ); + configCacheNotifier = new CommonCacheNotifier( + initAuthenticatorConfigMap(authenticatorMapper), + discoveryProvider, + httpClient, + "/druid-ext/basic-security/authentication/listen/config/%s", + "CoordinatorBasicAuthenticatorCacheNotifier" + ); } @LifecycleStart public void start() { if (!lifecycleLock.canStart()) { - throw new ISE("can't start."); + throw new ISE("Can't start."); } try { - cacheNotifier.start(); + userCacheNotifier.start(); + configCacheNotifier.start(); lifecycleLock.started(); } finally { @@ -85,7 +93,8 @@ public void stop() return; } try { - cacheNotifier.stop(); + userCacheNotifier.stop(); + configCacheNotifier.stop(); } finally { lifecycleLock.exitStop(); @@ -93,10 +102,17 @@ public void stop() } @Override - public void addUpdate(String updatedAuthorizerPrefix, byte[] updatedUserMap) + public void addUserUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserMap) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + userCacheNotifier.addUpdate(updatedAuthenticatorPrefix, updatedUserMap); + } + + @Override + public void addConfigUpdate(String updatedAuthenticatorPrefix, byte[] updatedConfig) { Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - cacheNotifier.addUpdate(updatedAuthorizerPrefix, updatedUserMap); + configCacheNotifier.addUpdate(updatedAuthenticatorPrefix, updatedConfig); } private Map initAuthenticatorConfigMap(AuthenticatorMapper mapper) diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java index 2641280ed9d9..2c66cc5674e8 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java @@ -43,6 +43,7 @@ import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.BytesFullResponseHandler; import org.apache.druid.security.basic.authentication.BytesFullResponseHolder; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; import org.apache.druid.server.security.Authenticator; import org.apache.druid.server.security.AuthenticatorMapper; @@ -69,6 +70,7 @@ public class CoordinatorPollingBasicAuthenticatorCacheManager implements BasicAu private static final EmittingLogger LOG = new EmittingLogger(CoordinatorPollingBasicAuthenticatorCacheManager.class); private final ConcurrentHashMap> cachedUserMaps; + private final ConcurrentHashMap cachedConfigs; private final Set authenticatorPrefixes; private final Injector injector; private final ObjectMapper objectMapper; @@ -90,6 +92,7 @@ public CoordinatorPollingBasicAuthenticatorCacheManager( this.commonCacheConfig = commonCacheConfig; this.objectMapper = objectMapper; this.cachedUserMaps = new ConcurrentHashMap<>(); + this.cachedConfigs = new ConcurrentHashMap<>(); this.authenticatorPrefixes = new HashSet<>(); this.druidLeaderClient = druidLeaderClient; } @@ -101,7 +104,7 @@ public void start() throw new ISE("can't start."); } - LOG.info("Starting DefaultBasicAuthenticatorCacheManager."); + LOG.info("Starting CoordinatorPollingBasicAuthenticatorCacheManager."); try { initUserMaps(); @@ -113,17 +116,17 @@ public void start() () -> { try { long randomDelay = ThreadLocalRandom.current().nextLong(0, commonCacheConfig.getMaxRandomDelay()); - LOG.debug("Inserting random polling delay of [%s] ms", randomDelay); + LOG.debug("Inserting cachedUserMaps random polling delay of [%s] ms", randomDelay); Thread.sleep(randomDelay); - LOG.debug("Scheduled cache poll is running"); + LOG.debug("Scheduled user cache poll is running"); for (String authenticatorPrefix : authenticatorPrefixes) { Map userMap = fetchUserMapFromCoordinator(authenticatorPrefix, false); if (userMap != null) { cachedUserMaps.put(authenticatorPrefix, userMap); } } - LOG.debug("Scheduled cache poll is done"); + LOG.debug("Scheduled user cache poll is done"); } catch (Throwable t) { LOG.makeAlert(t, "Error occured while polling for cachedUserMaps.").emit(); @@ -131,8 +134,33 @@ public void start() } ); + ScheduledExecutors.scheduleWithFixedDelay( + exec, + new Duration(commonCacheConfig.getPollingPeriod()), + new Duration(commonCacheConfig.getPollingPeriod()), + () -> { + try { + long randomDelay = ThreadLocalRandom.current().nextLong(0, commonCacheConfig.getMaxRandomDelay()); + LOG.debug("Inserting cachedConfigs random polling delay of [%s] ms", randomDelay); + Thread.sleep(randomDelay); + + LOG.debug("Scheduled config cache poll is running"); + for (String authenticatorPrefix : authenticatorPrefixes) { + BasicAuthConfig config = fetchConfigFromCoordinator(authenticatorPrefix); + if (config != null) { + cachedConfigs.put(authenticatorPrefix, config); + } + } + LOG.debug("Scheduled config cache poll is done"); + } + catch (Throwable t) { + LOG.makeAlert(t, "Error occured while polling for cachedConfigs.").emit(); + } + } + ); + lifecycleLock.started(); - LOG.info("Started DefaultBasicAuthenticatorCacheManager."); + LOG.info("Started CoordinatorPollingBasicAuthenticatorCacheManager."); } finally { lifecycleLock.exitStart(); @@ -146,15 +174,15 @@ public void stop() throw new ISE("can't stop."); } - LOG.info("DefaultBasicAuthenticatorCacheManager is stopping."); + LOG.info("CoordinatorPollingBasicAuthenticatorCacheManager is stopping."); exec.shutdown(); - LOG.info("DefaultBasicAuthenticatorCacheManager is stopped."); + LOG.info("CoordinatorPollingBasicAuthenticatorCacheManager is stopped."); } @Override - public void handleAuthenticatorUpdate(String authenticatorPrefix, byte[] serializedUserMap) + public void handleAuthenticatorUserMapUpdate(String authenticatorPrefix, byte[] serializedUserMap) { - LOG.debug("Received cache update for authenticator [%s].", authenticatorPrefix); + LOG.debug("Received user cache update for authenticator [%s].", authenticatorPrefix); Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); try { cachedUserMaps.put( @@ -170,7 +198,7 @@ public void handleAuthenticatorUpdate(String authenticatorPrefix, byte[] seriali } } catch (Exception e) { - LOG.makeAlert(e, "WTF? Could not deserialize user map received from coordinator.").emit(); + LOG.makeAlert(e, "Could not deserialize user map received from coordinator.").emit(); } } @@ -182,6 +210,33 @@ public Map getUserMap(String authenticatorPrefix return cachedUserMaps.get(authenticatorPrefix); } + @Override + public void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig) + { + LOG.debug("Received config cache update for authenticator [%s].", authenticatorPrefix); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + try { + cachedConfigs.put( + authenticatorPrefix, + objectMapper.readValue( + serializedConfig, + BasicAuthConfig.class + ) + ); + } + catch (Exception e) { + LOG.makeAlert(e, "Could not deserialize config received from coordinator.").emit(); + } + } + + @Override + public BasicAuthConfig getConfig(String authenticatorPrefix) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + + return cachedConfigs.get(authenticatorPrefix); + } + @Nullable private Map fetchUserMapFromCoordinator(String prefix, boolean isInit) { @@ -213,6 +268,24 @@ private Map fetchUserMapFromCoordinator(String p } } + @Nullable + private BasicAuthConfig fetchConfigFromCoordinator(String prefix) + { + try { + return RetryUtils.retry( + () -> { + return tryFetchConfigFromCoordinator(prefix); + }, + e -> true, + commonCacheConfig.getMaxSyncRetries() + ); + } + catch (Exception e) { + LOG.makeAlert(e, "Encountered exception while fetching config for authenticator [%s]", prefix).emit(); + return null; + } + } + private String getUserMapFilename(String prefix) { return StringUtils.format("%s.authenticator.cache", prefix); @@ -266,6 +339,20 @@ private Map tryFetchUserMapFromCoordinator(Strin return userMap; } + private BasicAuthConfig tryFetchConfigFromCoordinator(String prefix) throws Exception + { + Request req = druidLeaderClient.makeRequest( + HttpMethod.GET, + StringUtils.format("/druid-ext/basic-security/authentication/db/%s/cachedSerializedConfig", prefix) + ); + BytesFullResponseHolder responseHolder = (BytesFullResponseHolder) druidLeaderClient.go( + req, + new BytesFullResponseHandler() + ); + byte[] configBytes = responseHolder.getBytes(); + return objectMapper.readValue(configBytes, BasicAuthConfig.class); + } + private void initUserMaps() { AuthenticatorMapper authenticatorMapper = injector.getInstance(AuthenticatorMapper.class); @@ -279,10 +366,16 @@ private void initUserMaps() if (authenticator instanceof BasicHTTPAuthenticator) { String authenticatorName = entry.getKey(); authenticatorPrefixes.add(authenticatorName); + Map userMap = fetchUserMapFromCoordinator(authenticatorName, true); if (userMap != null) { cachedUserMaps.put(authenticatorName, userMap); } + + BasicAuthConfig config = fetchConfigFromCoordinator(authenticatorName); + if (config != null) { + cachedConfigs.put(authenticatorName, config); + } } } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/MetadataStoragePollingBasicAuthenticatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/MetadataStoragePollingBasicAuthenticatorCacheManager.java index 1a9bfcc02adf..d062404852ca 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/MetadataStoragePollingBasicAuthenticatorCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/MetadataStoragePollingBasicAuthenticatorCacheManager.java @@ -22,6 +22,7 @@ import com.google.inject.Inject; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.security.basic.authentication.db.updater.BasicAuthenticatorMetadataStorageUpdater; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; import java.util.Map; @@ -47,7 +48,7 @@ public MetadataStoragePollingBasicAuthenticatorCacheManager( } @Override - public void handleAuthenticatorUpdate(String authenticatorPrefix, byte[] serializedUserMap) + public void handleAuthenticatorUserMapUpdate(String authenticatorPrefix, byte[] serializedUserMap) { } @@ -56,4 +57,15 @@ public Map getUserMap(String authenticatorPrefix { return storageUpdater.getCachedUserMap(authenticatorPrefix); } + + @Override + public void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig) + { + } + + @Override + public BasicAuthConfig getConfig(String authenticatorPrefix) + { + return storageUpdater.getCachedConfig(authenticatorPrefix); + } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/BasicAuthenticatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/BasicAuthenticatorMetadataStorageUpdater.java index d354694458ff..a16ffe4df310 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/BasicAuthenticatorMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/BasicAuthenticatorMetadataStorageUpdater.java @@ -19,6 +19,7 @@ package org.apache.druid.security.basic.authentication.db.updater; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -45,4 +46,12 @@ public interface BasicAuthenticatorMetadataStorageUpdater byte[] getCurrentUserMapBytes(String prefix); void refreshAllNotification(); + + void updateConfig(String prefix, BasicAuthConfig config); + + BasicAuthConfig getCachedConfig(String prefix); + + byte[] getCachedSerializedConfig(String prefix); + + byte[] getCurrentConfigBytes(String prefix); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java index 941b4a38a0e4..54c36f68d4ec 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java @@ -42,6 +42,8 @@ import org.apache.druid.security.basic.BasicSecurityDBResourceException; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheNotifier; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; +import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorConfigBundle; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -66,6 +68,7 @@ public class CoordinatorBasicAuthenticatorMetadataStorageUpdater implements Basi private static final EmittingLogger LOG = new EmittingLogger(CoordinatorBasicAuthenticatorMetadataStorageUpdater.class); private static final String USERS = "users"; + private static final String CONFIG = "config"; private static final long UPDATE_RETRY_DELAY = 1000; private final AuthenticatorMapper authenticatorMapper; @@ -77,6 +80,7 @@ public class CoordinatorBasicAuthenticatorMetadataStorageUpdater implements Basi private final int numRetries = 5; private final Map cachedUserMaps; + private final Map cachedConfigMaps; private final Set authenticatorPrefixes; private final LifecycleLock lifecycleLock = new LifecycleLock(); @@ -102,6 +106,7 @@ public CoordinatorBasicAuthenticatorMetadataStorageUpdater( this.objectMapper = objectMapper; this.cacheNotifier = cacheNotifier; this.cachedUserMaps = new ConcurrentHashMap<>(); + this.cachedConfigMaps = new ConcurrentHashMap<>(); this.authenticatorPrefixes = new HashSet<>(); } @@ -156,6 +161,9 @@ public void start() ) ); } + + BasicAuthConfig config = new BasicAuthConfig(dbConfig); + updateConfigInternal(authenticatorName, config); } } @@ -172,7 +180,7 @@ public ScheduledExecutors.Signal call() return ScheduledExecutors.Signal.STOP; } try { - LOG.debug("Scheduled db poll is running"); + LOG.debug("Scheduled db userMap poll is running"); for (String authenticatorPrefix : authenticatorPrefixes) { byte[] userMapBytes = getCurrentUserMapBytes(authenticatorPrefix); @@ -184,7 +192,7 @@ public ScheduledExecutors.Signal call() cachedUserMaps.put(authenticatorPrefix, new BasicAuthenticatorUserMapBundle(userMap, userMapBytes)); } } - LOG.debug("Scheduled db poll is done"); + LOG.debug("Scheduled db userMap poll is done"); } catch (Throwable t) { LOG.makeAlert(t, "Error occured while polling for cachedUserMaps.").emit(); @@ -194,6 +202,43 @@ public ScheduledExecutors.Signal call() } ); + ScheduledExecutors.scheduleWithFixedDelay( + exec, + new Duration(commonCacheConfig.getPollingPeriod()), + new Duration(commonCacheConfig.getPollingPeriod()), + new Callable() + { + @Override + public ScheduledExecutors.Signal call() + { + if (stopped) { + return ScheduledExecutors.Signal.STOP; + } + try { + LOG.debug("Scheduled db config poll is running"); + for (String authenticatorPrefix : authenticatorPrefixes) { + + byte[] configBytes = getCurrentConfigBytes(authenticatorPrefix); + + if (configBytes != null) { + BasicAuthConfig config = BasicAuthUtils.deserializeAuthenticatorConfig( + objectMapper, + configBytes + ); + if (config != null) { + cachedConfigMaps.put(authenticatorPrefix, new BasicAuthenticatorConfigBundle(config, configBytes)); + } + } + } + LOG.debug("Scheduled db config poll is done"); + } + catch (Throwable t) { + LOG.makeAlert(t, "Error occured while polling for cachedConfigMaps.").emit(); + } + return ScheduledExecutors.Signal.REPEAT; + } + } + ); lifecycleLock.started(); } finally { @@ -276,50 +321,66 @@ public void refreshAllNotification() { cachedUserMaps.forEach( (authenticatorName, userMapBundle) -> { - cacheNotifier.addUpdate(authenticatorName, userMapBundle.getSerializedUserMap()); + cacheNotifier.addUserUpdate(authenticatorName, userMapBundle.getSerializedUserMap()); + } + ); + cachedConfigMaps.forEach( + (authenticatorName, configBundle) -> { + cacheNotifier.addConfigUpdate(authenticatorName, configBundle.getSerializedConfig()); } ); } - private static String getPrefixedKeyColumn(String keyPrefix, String keyName) + @Override + public void updateConfig(String prefix, BasicAuthConfig config) { - return StringUtils.format("basic_authentication_%s_%s", keyPrefix, keyName); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + + updateConfigInternal(prefix, config); } - private boolean tryUpdateUserMap( - String prefix, - Map userMap, - byte[] oldValue, - byte[] newValue - ) + @Override + public BasicAuthConfig getCachedConfig(String prefix) { - try { - MetadataCASUpdate update = new MetadataCASUpdate( - connectorConfig.getConfigTable(), - MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, - MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, - getPrefixedKeyColumn(prefix, USERS), - oldValue, - newValue - ); - - boolean succeeded = connector.compareAndSwap( - Collections.singletonList(update) - ); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - if (succeeded) { - cachedUserMaps.put(prefix, new BasicAuthenticatorUserMapBundle(userMap, newValue)); - cacheNotifier.addUpdate(prefix, newValue); - return true; - } else { - return false; - } + BasicAuthenticatorConfigBundle bundle = cachedConfigMaps.get(prefix); + if (bundle == null) { + return null; + } else { + return bundle.getConfig(); } - catch (Exception e) { - throw new RuntimeException(e); + } + + @Override + public byte[] getCachedSerializedConfig(String prefix) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + + BasicAuthenticatorConfigBundle bundle = cachedConfigMaps.get(prefix); + if (bundle == null) { + return null; + } else { + return bundle.getSerializedConfig(); } } + @Override + public byte[] getCurrentConfigBytes(String prefix) + { + return connector.lookup( + connectorConfig.getConfigTable(), + MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, + MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, + getPrefixedKeyColumn(prefix, CONFIG) + ); + } + + private static String getPrefixedKeyColumn(String keyPrefix, String keyName) + { + return StringUtils.format("basic_authentication_%s_%s", keyPrefix, keyName); + } + private void createUserInternal(String prefix, String userName) { int attempts = 0; @@ -394,6 +455,25 @@ private void setUserCredentialsInternal(String prefix, String userName, BasicAut throw new ISE("Could not set credentials for user[%s] due to concurrent update contention.", userName); } + private void updateConfigInternal(String prefix, BasicAuthConfig config) + { + int attempts = 0; + while (attempts < numRetries) { + if (updateConfigOnce(prefix, config)) { + return; + } else { + attempts++; + } + try { + Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); + } + catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + throw new ISE("Could not update config due to concurrent update contention."); + } + private boolean createUserOnce(String prefix, String userName) { byte[] oldValue = getCurrentUserMapBytes(prefix); @@ -441,4 +521,79 @@ private boolean setUserCredentialOnce(String prefix, String userName, BasicAuthe byte[] newValue = BasicAuthUtils.serializeAuthenticatorUserMap(objectMapper, userMap); return tryUpdateUserMap(prefix, userMap, oldValue, newValue); } + + private boolean updateConfigOnce(String prefix, BasicAuthConfig config) + { + byte[] oldValue = getCurrentConfigBytes(prefix); + byte[] newValue = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); + return tryUpdateConfig(prefix, config, oldValue, newValue); + } + + private boolean tryUpdateUserMap( + String prefix, + Map userMap, + byte[] oldValue, + byte[] newValue + ) + { + try { + MetadataCASUpdate update = new MetadataCASUpdate( + connectorConfig.getConfigTable(), + MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, + MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, + getPrefixedKeyColumn(prefix, USERS), + oldValue, + newValue + ); + + boolean succeeded = connector.compareAndSwap( + Collections.singletonList(update) + ); + + if (succeeded) { + cachedUserMaps.put(prefix, new BasicAuthenticatorUserMapBundle(userMap, newValue)); + cacheNotifier.addUserUpdate(prefix, newValue); + return true; + } else { + return false; + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private boolean tryUpdateConfig( + String prefix, + BasicAuthConfig config, + byte[] oldValue, + byte[] newValue + ) + { + try { + MetadataCASUpdate update = new MetadataCASUpdate( + connectorConfig.getConfigTable(), + MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, + MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, + getPrefixedKeyColumn(prefix, CONFIG), + oldValue, + newValue + ); + + boolean succeeded = connector.compareAndSwap( + Collections.singletonList(update) + ); + + if (succeeded) { + cachedConfigMaps.put(prefix, new BasicAuthenticatorConfigBundle(config, newValue)); + cacheNotifier.addConfigUpdate(prefix, newValue); + return true; + } else { + return false; + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResource.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResource.java index 5dc1493ac6b0..af2d7382ecf0 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResource.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResource.java @@ -24,6 +24,7 @@ import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.guice.LazySingleton; import org.apache.druid.security.basic.BasicSecurityResourceFilter; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import javax.servlet.http.HttpServletRequest; @@ -211,7 +212,65 @@ public Response getCachedSerializedUserMap( } /** - * Listen for update notifications for the auth storage + * @param req HTTP request + * + * @return Authenticator configuration + */ + @GET + @Path("/db/{authenticatorName}/config") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response getConfig( + @Context HttpServletRequest req, + @PathParam("authenticatorName") final String authenticatorName + ) + { + return handler.getConfig(authenticatorName); + } + + /** + * Update authenticator configuration + * + * @param req HTTP request + * @param config Authenticator configuration + * + * @return OK response + */ + @POST + @Path("/db/{authenticatorName}/config") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response updateConfig( + @Context HttpServletRequest req, + @PathParam("authenticatorName") final String authenticatorName, + BasicAuthConfig config + ) + { + return handler.updateConfig(authenticatorName, config); + } + + /** + * @param req HTTP request + * + * @return serialized config + */ + @GET + @Path("/db/{authenticatorName}/cachedSerializedConfig") + @Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response getCachedSerializedConfig( + @Context HttpServletRequest req, + @PathParam("authenticatorName") final String authenticatorName + ) + { + return handler.getCachedSerializedConfig(authenticatorName); + } + + /** + * Listen for users update notifications for the auth storage */ @POST @Path("/listen/{authenticatorName}") @@ -224,6 +283,23 @@ public Response authenticatorUpdateListener( byte[] serializedUserMap ) { - return handler.authenticatorUpdateListener(authenticatorName, serializedUserMap); + return handler.authenticatorUserUpdateListener(authenticatorName, serializedUserMap); + } + + /** + * Listen for config update notifications for the auth storage + */ + @POST + @Path("/listen/config/{authenticatorName}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response authenticatorConfigUpdateListener( + @Context HttpServletRequest req, + @PathParam("authenticatorName") final String authenticatorName, + byte[] serializedConfig + ) + { + return handler.authenticatorConfigUpdateListener(authenticatorName, serializedConfig); } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResourceHandler.java index 35f0d4d405b3..24396b110a30 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResourceHandler.java @@ -19,6 +19,7 @@ package org.apache.druid.security.basic.authentication.endpoint; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import javax.ws.rs.core.Response; @@ -43,10 +44,18 @@ public interface BasicAuthenticatorResourceHandler Response getCachedSerializedUserMap(String authenticatorName); + Response getConfig(String authenticatorName); + + Response updateConfig(String authenticatorName, BasicAuthConfig config); + + Response getCachedSerializedConfig(String authenticatorName); + Response refreshAll(); // non-coordinator methods - Response authenticatorUpdateListener(String authenticatorName, byte[] serializedUserMap); + Response authenticatorUserUpdateListener(String authenticatorName, byte[] serializedUserMap); + + Response authenticatorConfigUpdateListener(String authenticatorName, byte[] serializedConfig); // common methods Response getLoadStatus(); diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java index 714e3da4e723..48081d6ae2d4 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java @@ -28,6 +28,7 @@ import org.apache.druid.security.basic.BasicSecurityDBResourceException; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.updater.BasicAuthenticatorMetadataStorageUpdater; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; import org.apache.druid.server.security.Authenticator; @@ -179,7 +180,70 @@ public Response refreshAll() } @Override - public Response authenticatorUpdateListener(String authenticatorName, byte[] serializedUserMap) + public Response getConfig(String authenticatorName) + { + final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); + if (authenticator == null) { + return makeResponseForAuthenticatorNotFound(authenticatorName); + } + + BasicAuthConfig config = BasicAuthUtils.deserializeAuthenticatorConfig( + objectMapper, + storageUpdater.getCurrentConfigBytes(authenticatorName) + ); + if (config == null) { + throw new BasicSecurityDBResourceException("Authenticator [%s] config does not exist.", authenticatorName); + } + + BasicAuthConfig maskedConfig = new BasicAuthConfig( + config.getUrl(), + config.getBindUser(), + config.getBindPassword() != null ? "..." : null, + config.getBaseDn(), + config.getUserSearch(), + config.getUserAttribute(), + config.getGroupFilters() + ); + + return Response.ok(maskedConfig).build(); + } + + @Override + public Response updateConfig(String authenticatorName, BasicAuthConfig config) + { + final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); + if (authenticator == null) { + return makeResponseForAuthenticatorNotFound(authenticatorName); + } + + try { + storageUpdater.updateConfig(authenticatorName, config); + return Response.ok().build(); + } + catch (BasicSecurityDBResourceException cfe) { + return makeResponseForBasicSecurityDBResourceException(cfe); + } + } + + @Override + public Response getCachedSerializedConfig(String authenticatorName) + { + final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); + if (authenticator == null) { + return makeResponseForAuthenticatorNotFound(authenticatorName); + } + + return Response.ok(storageUpdater.getCachedSerializedConfig(authenticatorName)).build(); + } + + @Override + public Response authenticatorUserUpdateListener(String authenticatorName, byte[] serializedUserMap) + { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + @Override + public Response authenticatorConfigUpdateListener(String authenticatorName, byte[] serializedConfig) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -190,7 +254,8 @@ public Response getLoadStatus() Map loadStatus = new HashMap<>(); authenticatorMap.forEach( (authenticatorName, authenticator) -> { - loadStatus.put(authenticatorName, storageUpdater.getCachedUserMap(authenticatorName) != null); + loadStatus.put(authenticatorName, storageUpdater.getCachedUserMap(authenticatorName) != null && + storageUpdater.getCachedConfig(authenticatorName) != null); } ); return Response.ok(loadStatus).build(); diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java index 4f196f719122..6d232a2145fa 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java @@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.server.security.Authenticator; import org.apache.druid.server.security.AuthenticatorMapper; @@ -109,7 +110,44 @@ public Response refreshAll() } @Override - public Response authenticatorUpdateListener(String authenticatorName, byte[] serializedUserMap) + public Response getConfig(String authenticatorName) + { + return NOT_FOUND_RESPONSE; + } + + @Override + public Response updateConfig(String authenticatorName, BasicAuthConfig config) + { + return NOT_FOUND_RESPONSE; + } + + @Override + public Response getCachedSerializedConfig(String authenticatorName) + { + return NOT_FOUND_RESPONSE; + } + + @Override + public Response authenticatorUserUpdateListener(String authenticatorName, byte[] serializedUserMap) + { + final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); + if (authenticator == null) { + String errMsg = StringUtils.format("Received update for unknown authenticator[%s]", authenticatorName); + log.error(errMsg); + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of( + "error", + StringUtils.format(errMsg) + )) + .build(); + } + + cacheManager.handleAuthenticatorUserMapUpdate(authenticatorName, serializedUserMap); + return Response.ok().build(); + } + + @Override + public Response authenticatorConfigUpdateListener(String authenticatorName, byte[] serializedConfig) { final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); if (authenticator == null) { @@ -123,7 +161,7 @@ public Response authenticatorUpdateListener(String authenticatorName, byte[] ser .build(); } - cacheManager.handleAuthenticatorUpdate(authenticatorName, serializedUserMap); + cacheManager.handleAuthenticatorConfigUpdate(authenticatorName, serializedConfig); return Response.ok().build(); } @@ -133,7 +171,9 @@ public Response getLoadStatus() Map loadStatus = new HashMap<>(); authenticatorMap.forEach( (authenticatorName, authenticator) -> { - loadStatus.put(authenticatorName, cacheManager.getUserMap(authenticatorName) != null); + loadStatus.put(authenticatorName, cacheManager.getUserMap(authenticatorName) != null && + cacheManager.getConfig(authenticatorName) != null + ); } ); return Response.ok(loadStatus).build(); diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthConfig.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthConfig.java new file mode 100644 index 000000000000..2aea9c6548b2 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthConfig.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authentication.entity; + +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.security.basic.BasicAuthDBConfig; + +@JsonInclude(JsonInclude.Include.NON_NULL) +public class BasicAuthConfig +{ + private final String url; + private final String bindUser; + private final String bindPassword; + private final String baseDn; + private final String userSearch; + private final String userAttribute; + private final String[] groupFilters; + + public BasicAuthConfig( + @JsonProperty("url") String url, + @JsonProperty("bindUser") String bindUser, + @JsonProperty("bindPassword") String bindPassword, + @JsonProperty("baseDn") String baseDn, + @JsonProperty("userSearch") String userSearch, + @JsonProperty("userAttribute") String userAttribute, + @JsonProperty("groupFilters") String[] groupFilters + ) + { + this.url = url; + this.bindUser = bindUser; + this.bindPassword = bindPassword; + this.baseDn = baseDn; + this.userSearch = userSearch; + this.userAttribute = userAttribute; + this.groupFilters = groupFilters; + } + + public BasicAuthConfig(BasicAuthDBConfig config) + { + this.url = config.getUrl(); + this.bindUser = config.getBindUser(); + this.bindPassword = config.getBindPassword() != null ? config.getBindPassword().getPassword() : null; + this.baseDn = config.getBaseDn(); + this.userSearch = config.getUserSearch(); + this.userAttribute = config.getUserAttribute(); + this.groupFilters = config.getGroupFilters(); + } + + @JsonProperty + public String getUrl() + { + return url; + } + + @JsonProperty + public String getBindUser() + { + return bindUser; + } + + @JsonProperty + public String getBindPassword() + { + return bindPassword; + } + + @JsonProperty + public String getBaseDn() + { + return baseDn; + } + + @JsonProperty + public String getUserSearch() + { + return userSearch; + } + + @JsonProperty + public String getUserAttribute() + { + return userAttribute; + } + + @JsonProperty + public String[] getGroupFilters() + { + return groupFilters; + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorConfigBundle.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorConfigBundle.java new file mode 100644 index 000000000000..b6dd49297185 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorConfigBundle.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authentication.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class BasicAuthenticatorConfigBundle +{ + private final BasicAuthConfig config; + private final byte[] serializedConfig; + + @JsonCreator + public BasicAuthenticatorConfigBundle( + @JsonProperty("config") BasicAuthConfig config, + @JsonProperty("serializedConfig") byte[] serializedConfig + ) + { + this.config = config; + this.serializedConfig = serializedConfig; + } + + @JsonProperty + public BasicAuthConfig getConfig() + { + return config; + } + + @JsonProperty + public byte[] getSerializedConfig() + { + return serializedConfig; + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/CredentialsValidator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/CredentialsValidator.java new file mode 100644 index 000000000000..a7e3efd1d647 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/CredentialsValidator.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authentication.validator; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import org.apache.druid.server.security.AuthenticationResult; + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DBCredentialsValidator.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "db", value = DBCredentialsValidator.class), + @JsonSubTypes.Type(name = "ldap", value = LDAPCredentialsValidator.class), +}) +public interface CredentialsValidator +{ + AuthenticationResult validateCredentials( + String authenticatorName, + String authorizerName, + String username, + char[] password + ); +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/DBCredentialsValidator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/DBCredentialsValidator.java new file mode 100644 index 000000000000..11af6925977b --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/DBCredentialsValidator.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authentication.validator; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.inject.Provider; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.security.basic.BasicAuthUtils; +import org.apache.druid.security.basic.BasicSecurityAuthenticationException; +import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; +import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; +import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; +import org.apache.druid.server.security.AuthenticationResult; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.Map; + +@JsonTypeName("db") +public class DBCredentialsValidator implements CredentialsValidator +{ + private static final Logger LOG = new Logger(DBCredentialsValidator.class); + private final Provider cacheManager; + + @JsonCreator + public DBCredentialsValidator( + @JacksonInject Provider cacheManager + ) + { + this.cacheManager = cacheManager; + } + + @Override + @Nullable + public AuthenticationResult validateCredentials( + String authenticatorName, + String authorizerName, + String username, + char[] password + ) + { + Map userMap = cacheManager.get().getUserMap(authenticatorName); + if (userMap == null) { + throw new IAE("No authenticator found with prefix: [%s]", authenticatorName); + } + + BasicAuthenticatorUser user = userMap.get(username); + if (user == null) { + return null; + } + BasicAuthenticatorCredentials credentials = user.getCredentials(); + if (credentials == null) { + return null; + } + + byte[] recalculatedHash = BasicAuthUtils.hashPassword( + password, + credentials.getSalt(), + credentials.getIterations() + ); + + if (Arrays.equals(recalculatedHash, credentials.getHash())) { + return new AuthenticationResult(username, authorizerName, authenticatorName, null); + } else { + throw new BasicSecurityAuthenticationException("User DB authentication failed username[%s].", username); + } + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java new file mode 100644 index 000000000000..6515fa4ce784 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java @@ -0,0 +1,422 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authentication.validator; + +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonTypeName; +import com.google.inject.Provider; +import org.apache.druid.java.util.common.RE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.metadata.DefaultPasswordProvider; +import org.apache.druid.metadata.PasswordProvider; +import org.apache.druid.security.basic.BasicAuthDBConfig; +import org.apache.druid.security.basic.BasicAuthUtils; +import org.apache.druid.security.basic.BasicSecurityAuthenticationException; +import org.apache.druid.security.basic.BasicSecuritySSLSocketFactory; +import org.apache.druid.security.basic.authentication.BasicAuthenticatorUserPrincipal; +import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; +import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; +import org.apache.druid.server.security.AuthenticationResult; + +import javax.annotation.Nullable; +import javax.naming.AuthenticationException; +import javax.naming.Context; +import javax.naming.InvalidNameException; +import javax.naming.NamingEnumeration; +import javax.naming.NamingException; +import javax.naming.directory.Attribute; +import javax.naming.directory.DirContext; +import javax.naming.directory.InitialDirContext; +import javax.naming.directory.SearchControls; +import javax.naming.directory.SearchResult; +import javax.naming.ldap.LdapName; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.ReentrantLock; + +@JsonTypeName("ldap") +public class LDAPCredentialsValidator implements CredentialsValidator +{ + private static final Logger LOG = new Logger(LDAPCredentialsValidator.class); + private static final ReentrantLock LOCK = new ReentrantLock(); + + private final LruBlockCache cache; + private final BasicAuthenticatorCacheManager cacheManager; + + private AtomicReference dbConfig = new AtomicReference<>(); + + @JsonCreator + public LDAPCredentialsValidator( + @JacksonInject Provider cacheManager, + @JsonProperty("url") String url, + @JsonProperty("bindUser") String bindUser, + @JsonProperty("bindPassword") PasswordProvider bindPassword, + @JsonProperty("baseDn") String baseDn, + @JsonProperty("userSearch") String userSearch, + @JsonProperty("userAttribute") String userAttribute, + @JsonProperty("groupFilters") String[] groupFilters, + @JsonProperty("credentialIterations") Integer credentialIterations, + @JsonProperty("credentialVerifyDuration") Integer credentialVerifyDuration, + @JsonProperty("credentialMaxDuration") Integer credentialMaxDuration, + @JsonProperty("credentialCacheSize") Integer credentialCacheSize + ) + { + this.cacheManager = cacheManager.get(); + this.dbConfig.set(new BasicAuthDBConfig( + null, + null, + null, + null, + null, + true, + BasicAuthDBConfig.DEFAULT_CACHE_NOTIFY_TIMEOUT_MS, + credentialIterations == null ? BasicAuthUtils.DEFAULT_KEY_ITERATIONS : credentialIterations, + url, + bindUser, + bindPassword, + baseDn, + userSearch, + userAttribute, + groupFilters, + credentialVerifyDuration == null ? BasicAuthUtils.DEFAULT_CREDENTIAL_VERIFY_DURATION_SECONDS : credentialVerifyDuration, + credentialMaxDuration == null ? BasicAuthUtils.DEFAULT_CREDENTIAL_MAX_DURATION_SECONDS : credentialMaxDuration, + credentialCacheSize == null ? BasicAuthUtils.DEFAULT_CREDENTIAL_CACHE_SIZE : credentialCacheSize + )); + this.cache = new LruBlockCache( + this.dbConfig.get().getCredentialCacheSize(), + this.dbConfig.get().getCredentialVerifyDuration(), + this.dbConfig.get().getCredentialMaxDuration() + ); + } + + Properties bindProperties(BasicAuthDBConfig dbConfig) + { + Properties properties = commonProperties(dbConfig); + properties.put(Context.SECURITY_PRINCIPAL, dbConfig.getBindUser()); + properties.put(Context.SECURITY_CREDENTIALS, dbConfig.getBindPassword().getPassword()); + return properties; + } + + Properties userProperties(BasicAuthDBConfig dbConfig, LdapName userDn, char[] password) + { + Properties properties = commonProperties(dbConfig); + properties.put(Context.SECURITY_PRINCIPAL, userDn.toString()); + properties.put(Context.SECURITY_CREDENTIALS, String.valueOf(password)); + return properties; + } + + Properties commonProperties(BasicAuthDBConfig dbConfig) + { + Properties properties = new Properties(); + properties.put(Context.INITIAL_CONTEXT_FACTORY, "com.sun.jndi.ldap.LdapCtxFactory"); + properties.put(Context.PROVIDER_URL, dbConfig.getUrl()); + properties.put(Context.SECURITY_AUTHENTICATION, "simple"); + if (StringUtils.toLowerCase(dbConfig.getUrl()).startsWith("ldaps://")) { + properties.put(Context.SECURITY_PROTOCOL, "ssl"); + properties.put("java.naming.ldap.factory.socket", BasicSecuritySSLSocketFactory.class.getName()); + } + return properties; + } + + @Override + public AuthenticationResult validateCredentials( + String authenticatorName, + String authorizerName, + String username, + char[] password + ) + { + updateConfig(authenticatorName); + BasicAuthDBConfig currentDBConfig = this.dbConfig.get(); + Set groups; + LdapName userDn; + Map contexMap = new HashMap<>(); + + BasicAuthenticatorUserPrincipal principal = this.cache.getOrExpire(username); + if (principal != null && principal.hasSameCredentials(password)) { + contexMap.put(BasicAuthUtils.GROUPS_CONTEXT_KEY, principal.getGroups()); + return new AuthenticationResult(username, authorizerName, authenticatorName, contexMap); + } else { + try { + InitialDirContext dirContext = new InitialDirContext(bindProperties(currentDBConfig)); + + try { + SearchResult userResult = getLdapUserObject(currentDBConfig, dirContext, username); + if (userResult == null) { + LOG.debug("User not found: %s", username); + return null; + } + userDn = new LdapName(userResult.getNameInNamespace()); + groups = getGroupsFromLdap(currentDBConfig, userResult); + if (groups == null || groups.isEmpty()) { + LOG.debug("User is not mapped to any groups: %s", username); + return null; + } + } + finally { + try { + dirContext.close(); + } + catch (Exception ignored) { + // ignored + } + } + + } + catch (NamingException e) { + LOG.error(e, "Exception during user lookup"); + return null; + } + + if (!validatePassword(currentDBConfig, userDn, password)) { + LOG.debug("Password incorrect for user %s", username); + throw new BasicSecurityAuthenticationException("User LDAP authentication failed username[%s].", userDn.toString()); + } + + byte[] salt = BasicAuthUtils.generateSalt(); + byte[] hash = BasicAuthUtils.hashPassword(password, salt, currentDBConfig.getIterations()); + BasicAuthenticatorUserPrincipal newPrincipal = new BasicAuthenticatorUserPrincipal( + username, + new BasicAuthenticatorCredentials(salt, hash, currentDBConfig.getIterations()), + groups + ); + + this.cache.put(username, newPrincipal); + contexMap.put(BasicAuthUtils.GROUPS_CONTEXT_KEY, groups); + return new AuthenticationResult(username, authorizerName, authenticatorName, contexMap); + } + } + + @Nullable + SearchResult getLdapUserObject(BasicAuthDBConfig dbConfig, DirContext context, String username) + { + try { + SearchControls sc = new SearchControls(); + sc.setSearchScope(SearchControls.SUBTREE_SCOPE); + sc.setReturningAttributes(new String[] {dbConfig.getUserAttribute(), "memberOf" }); + NamingEnumeration results = context.search( + dbConfig.getBaseDn(), + StringUtils.format(dbConfig.getUserSearch(), username), + sc); + try { + if (!results.hasMore()) { + return null; + } + return results.next(); + } + finally { + results.close(); + } + } + catch (NamingException e) { + LOG.debug(e, "Unable to find user '%s'", username); + return null; + } + } + + Set getGroupsFromLdap(BasicAuthDBConfig dbConfig, SearchResult userResult) throws NamingException + { + Set groups = new TreeSet<>(); + + Attribute memberOf = userResult.getAttributes().get("memberOf"); + if (memberOf == null) { + LOG.debug("No memberOf attributes"); + return groups; // not part of any groups + } + + Set groupFilters = new TreeSet<>(Arrays.asList(dbConfig.getGroupFilters())); + for (int i = 0; i < memberOf.size(); i++) { + String memberDn = memberOf.get(i).toString(); + LdapName ln; + try { + ln = new LdapName(memberDn); + } + catch (InvalidNameException e) { + LOG.debug("Invalid LDAP name: %s", memberDn); + continue; + } + + if (!allowedLdapGroup(ln, groupFilters)) { + continue; + } else { + groups.add(ln); + } + + // valid group name, get roles for it + // addGroups(groups, ln); + } + + return groups; + } + + boolean allowedLdapGroup(LdapName groupName, Set groupFilters) + { + for (String filter : groupFilters) { + try { + if (filter.startsWith("*,")) { + LdapName ln = new LdapName(filter.substring(2)); + if (groupName.startsWith(ln)) { + return true; + } + } else if (filter.endsWith(",*")) { + LdapName ln = new LdapName(filter.substring(0, filter.length() - 2)); + if (groupName.endsWith(ln)) { + return true; + } + } else { + LOG.debug("Attempting exact filter %s", filter); + LdapName ln = new LdapName(filter); + if (groupName.equals(ln)) { + return true; + } + } + } + catch (InvalidNameException e) { + throw new RE(StringUtils.format("Configuration problem - Invalid groupFilter '%s'", filter)); + } + } + return false; + } + + boolean validatePassword(BasicAuthDBConfig dbConfig, LdapName userDn, char[] password) + { + InitialDirContext context = null; + + try { + context = new InitialDirContext(userProperties(dbConfig, userDn, password)); + return true; + } + catch (AuthenticationException e) { + return false; + } + catch (NamingException e) { + LOG.error(e, "Exception during LDAP authentication username[%s]", userDn.toString()); + return false; + } + finally { + try { + if (context != null) { + context.close(); + } + } + catch (Exception ignored) { + // ignored + } + } + } + + private void updateConfig(String authenticatorName) + { + BasicAuthDBConfig cuurentDBConfig = this.dbConfig.get(); + BasicAuthConfig config = this.cacheManager.getConfig(authenticatorName); + + if (config != null) { + this.dbConfig.set(new BasicAuthDBConfig( + null, + null, + null, + null, + null, + cuurentDBConfig.isEnableCacheNotifications(), + cuurentDBConfig.getCacheNotificationTimeout(), + cuurentDBConfig.getIterations(), + config.getUrl(), + config.getBindUser(), + DefaultPasswordProvider.fromString(config.getBindPassword()), + config.getBaseDn(), + config.getUserSearch(), + config.getUserAttribute(), + config.getGroupFilters(), + cuurentDBConfig.getCredentialVerifyDuration(), + cuurentDBConfig.getCredentialMaxDuration(), + cuurentDBConfig.getCredentialCacheSize() + )); + } + } + + private static class LruBlockCache extends LinkedHashMap + { + + private static final long serialVersionUID = 7509410739092012261L; + + private final int cacheSize; + private final int duration; + private final int maxDuration; + + public LruBlockCache(int cacheSize, int duration, int maxDuration) + { + super(16, 0.75f, true); + this.cacheSize = cacheSize; + this.duration = duration; + this.maxDuration = maxDuration; + } + + @Override + protected boolean removeEldestEntry(Map.Entry eldest) + { + return size() > cacheSize; + } + + @Nullable + BasicAuthenticatorUserPrincipal getOrExpire(String identity) + { + try { + LOCK.lock(); + BasicAuthenticatorUserPrincipal principal = get(identity); + if (principal != null) { + if (principal.isExpired(duration, maxDuration)) { + remove(identity); + return null; + } else { + return principal; + } + } else { + return null; + } + } + finally { + LOCK.unlock(); + } + } + + @Override + public BasicAuthenticatorUserPrincipal put(String key, BasicAuthenticatorUserPrincipal value) + { + try { + LOCK.lock(); + return super.put(key, value); + } + finally { + LOCK.unlock(); + } + + } + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/BasicRoleBasedAuthorizer.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/BasicRoleBasedAuthorizer.java index 8e88ca9efe8b..04ae738fa68c 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/BasicRoleBasedAuthorizer.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/BasicRoleBasedAuthorizer.java @@ -25,7 +25,9 @@ import com.fasterxml.jackson.annotation.JsonTypeName; import org.apache.druid.java.util.common.IAE; import org.apache.druid.security.basic.BasicAuthDBConfig; +import org.apache.druid.security.basic.BasicAuthUtils; import org.apache.druid.security.basic.authorization.db.cache.BasicAuthorizerCacheManager; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerPermission; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; @@ -35,7 +37,13 @@ import org.apache.druid.server.security.Authorizer; import org.apache.druid.server.security.Resource; +import javax.naming.InvalidNameException; +import javax.naming.ldap.LdapName; +import java.util.HashSet; +import java.util.Locale; import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -51,6 +59,9 @@ public class BasicRoleBasedAuthorizer implements Authorizer public BasicRoleBasedAuthorizer( @JacksonInject BasicAuthorizerCacheManager cacheManager, @JsonProperty("name") String name, + @JsonProperty("initialAdminUser") String initialAdminUser, + @JsonProperty("initialAdminRole") String initialAdminRole, + @JsonProperty("initialAdminGroupMapping")String initialAdminGroupMapping, @JsonProperty("enableCacheNotifications") Boolean enableCacheNotifications, @JsonProperty("cacheNotificationTimeout") Long cacheNotificationTimeout ) @@ -60,9 +71,21 @@ public BasicRoleBasedAuthorizer( this.dbConfig = new BasicAuthDBConfig( null, null, + initialAdminUser, + initialAdminRole, + initialAdminGroupMapping, enableCacheNotifications == null ? true : enableCacheNotifications, cacheNotificationTimeout == null ? BasicAuthDBConfig.DEFAULT_CACHE_NOTIFY_TIMEOUT_MS : cacheNotificationTimeout, - 0 + 0, + null, + null, + null, + null, + null, + null, + null, null, + null, + null ); } @@ -70,29 +93,55 @@ public BasicRoleBasedAuthorizer( public Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action) { if (authenticationResult == null) { - throw new IAE("WTF? authenticationResult should never be null."); + throw new IAE("authenticationResult is null where it should never be."); } + Map roleMap; + Set roleNames = new HashSet<>(); + Map userMap = cacheManager.getUserMap(name); if (userMap == null) { throw new IAE("Could not load userMap for authorizer [%s]", name); } - Map roleMap = cacheManager.getRoleMap(name); - if (roleMap == null) { - throw new IAE("Could not load roleMap for authorizer [%s]", name); + Map groupMappingMap = cacheManager.getGroupMappingMap(name); + if (groupMappingMap == null) { + throw new IAE("Could not load groupMappingMap for authorizer [%s]", name); } BasicAuthorizerUser user = userMap.get(authenticationResult.getIdentity()); - if (user == null) { + if (user != null) { + roleNames.addAll(user.getRoles()); + roleMap = cacheManager.getRoleMap(name); + } else { + Set groupNamesFromLdap = Optional.ofNullable(authenticationResult.getContext()) + .map(contextMap -> contextMap.get(BasicAuthUtils.GROUPS_CONTEXT_KEY)) + .map(p -> { + if (p instanceof Set) { + return (Set) p; + } else { + return null; + } + }) + .orElse(new HashSet<>()); + + roleNames.addAll(getRoles(groupMappingMap, groupNamesFromLdap)); + roleMap = cacheManager.getGroupMappingRoleMap(name); + } + if (roleMap == null) { + throw new IAE("Could not load roleMap for authorizer [%s]", name); + } + if (roleNames.isEmpty()) { return new Access(false); } - for (String roleName : user.getRoles()) { + for (String roleName : roleNames) { BasicAuthorizerRole role = roleMap.get(roleName); - for (BasicAuthorizerPermission permission : role.getPermissions()) { - if (permissionCheck(resource, action, permission)) { - return new Access(true); + if (role != null) { + for (BasicAuthorizerPermission permission : role.getPermissions()) { + if (permissionCheck(resource, action, permission)) { + return new Access(true); + } } } } @@ -116,6 +165,45 @@ private boolean permissionCheck(Resource resource, Action action, BasicAuthorize return resourceNameMatcher.matches(); } + Set getRoles(Map groupMappingMap, Set groupNamesFromLdap) + { + Set roles = new HashSet<>(); + + if (groupMappingMap.size() == 0) { + return roles; + } + + for (LdapName groupName : groupNamesFromLdap) { + for (Map.Entry groupMappingEntry : groupMappingMap.entrySet()) { + BasicAuthorizerGroupMapping groupMapping = groupMappingEntry.getValue(); + String mask = groupMapping.getGroupPattern(); + try { + if (mask.startsWith("*,")) { + LdapName ln = new LdapName(mask.substring(2)); + if (groupName.startsWith(ln)) { + roles.addAll(groupMapping.getRoles()); + } + } else if (mask.endsWith(",*")) { + LdapName ln = new LdapName(mask.substring(0, mask.length() - 2)); + if (groupName.endsWith(ln)) { + roles.addAll(groupMapping.getRoles()); + } + } else { + LdapName ln = new LdapName(mask); + if (groupName.equals(ln)) { + roles.addAll(groupMapping.getRoles()); + } + } + } + catch (InvalidNameException e) { + throw new RuntimeException(String.format(Locale.getDefault(), + "Configuration problem - Invalid groupMapping '%s'", mask)); + } + } + } + return roles; + } + public BasicAuthDBConfig getDbConfig() { return dbConfig; diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/BasicAuthorizerCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/BasicAuthorizerCacheManager.java index ca2a0b9da7fa..7515d782f49a 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/BasicAuthorizerCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/BasicAuthorizerCacheManager.java @@ -19,6 +19,7 @@ package org.apache.druid.security.basic.authorization.db.cache; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; @@ -32,11 +33,18 @@ public interface BasicAuthorizerCacheManager { /** * Update this cache manager's local state with fresh information pushed by the coordinator. - * * @param authorizerPrefix The name of the authorizer this update applies to. * @param serializedUserAndRoleMap The updated, serialized user and role maps */ - void handleAuthorizerUpdate(String authorizerPrefix, byte[] serializedUserAndRoleMap); + void handleAuthorizerUserUpdate(String authorizerPrefix, byte[] serializedUserAndRoleMap); + + /** + * Update this cache manager's local state with fresh information pushed by the coordinator. + * @param authorizerPrefix The name of the authorizer this update applies to. + * @param serializedGroupMappingAndRoleMap The updated, serialized group and role maps + * */ + void handleAuthorizerGroupMappingUpdate(String authorizerPrefix, byte[] serializedGroupMappingAndRoleMap); + /** * Return the cache manager's local view of the user map for the authorizer named `authorizerPrefix`. @@ -53,4 +61,20 @@ public interface BasicAuthorizerCacheManager * @return Role map */ Map getRoleMap(String authorizerPrefix); + + /** + * Return the cache manager's local view of the groupMapping map for the authorizer named `authorizerPrefix`. + * + * @param authorizerPrefix The name of the authorizer + * @return GroupMapping map + */ + Map getGroupMappingMap(String authorizerPrefix); + + /** + * Return the cache manager's local view of the groupMapping-role map for the authorizer named `authorizerPrefix`. + * + * @param authorizerPrefix The name of the authorizer + * @return Role map + */ + Map getGroupMappingRoleMap(String authorizerPrefix); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/BasicAuthorizerCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/BasicAuthorizerCacheNotifier.java index 455f636e5ba1..e655e3bcc32c 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/BasicAuthorizerCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/BasicAuthorizerCacheNotifier.java @@ -26,9 +26,15 @@ public interface BasicAuthorizerCacheNotifier { /** * Send the user map state contained in updatedUserMap to all non-coordinator Druid services - * - * @param authorizerPrefix Name of authorizer being updated + * @param authorizerPrefix Name of authorizer being updated * @param userAndRoleMap User/role map state */ - void addUpdate(String authorizerPrefix, byte[] userAndRoleMap); + void addUpdateUser(String authorizerPrefix, byte[] userAndRoleMap); + + /** + * Send the groupMapping map state contained in updatedGroupMappingMap to all non-coordinator Druid services + * @param authorizerPrefix Name of authorizer being updated + * @param groupMappingAndRoleMap Group/role map state + */ + void addUpdateGroupMapping(String authorizerPrefix, byte[] groupMappingAndRoleMap); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java index 85c1a1caedc3..6693dbbdaba2 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java @@ -44,7 +44,8 @@ public class CoordinatorBasicAuthorizerCacheNotifier implements BasicAuthorizerC { private final LifecycleLock lifecycleLock = new LifecycleLock(); - private CommonCacheNotifier cacheNotifier; + private CommonCacheNotifier cacheUserNotifier; + private CommonCacheNotifier cacheGroupMappingNotifier; @Inject public CoordinatorBasicAuthorizerCacheNotifier( @@ -53,13 +54,20 @@ public CoordinatorBasicAuthorizerCacheNotifier( @EscalatedClient HttpClient httpClient ) { - cacheNotifier = new CommonCacheNotifier( + cacheUserNotifier = new CommonCacheNotifier( getAuthorizerConfigMap(authorizerMapper), discoveryProvider, httpClient, "/druid-ext/basic-security/authorization/listen/%s", "CoordinatorBasicAuthorizerCacheNotifier" ); + cacheGroupMappingNotifier = new CommonCacheNotifier( + getAuthorizerConfigMap(authorizerMapper), + discoveryProvider, + httpClient, + "/druid-ext/basic-security/authorization/listen/groupMappings/%s", + "CoordinatorBasicAuthorizerCacheNotifier" + ); } @LifecycleStart @@ -70,7 +78,8 @@ public void start() } try { - cacheNotifier.start(); + cacheUserNotifier.start(); + cacheGroupMappingNotifier.start(); lifecycleLock.started(); } finally { @@ -85,7 +94,8 @@ public void stop() return; } try { - cacheNotifier.stop(); + cacheUserNotifier.stop(); + cacheGroupMappingNotifier.stop(); } finally { lifecycleLock.exitStop(); @@ -93,10 +103,17 @@ public void stop() } @Override - public void addUpdate(String updatedAuthorizerPrefix, byte[] updatedUserMap) + public void addUpdateUser(String updatedAuthorizerPrefix, byte[] userAndRoleMap) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + cacheUserNotifier.addUpdate(updatedAuthorizerPrefix, userAndRoleMap); + } + + @Override + public void addUpdateGroupMapping(String updatedAuthorizerPrefix, byte[] groupMappingAndRoleMap) { Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - cacheNotifier.addUpdate(updatedAuthorizerPrefix, updatedUserMap); + cacheGroupMappingNotifier.addUpdate(updatedAuthorizerPrefix, groupMappingAndRoleMap); } private Map getAuthorizerConfigMap(AuthorizerMapper mapper) diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java index 29c3f572bc85..2085a63f60b8 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorPollingBasicAuthorizerCacheManager.java @@ -43,8 +43,10 @@ import org.apache.druid.security.basic.authentication.BytesFullResponseHandler; import org.apache.druid.security.basic.authentication.BytesFullResponseHolder; import org.apache.druid.security.basic.authorization.BasicRoleBasedAuthorizer; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; +import org.apache.druid.security.basic.authorization.entity.GroupMappingAndRoleMap; import org.apache.druid.security.basic.authorization.entity.UserAndRoleMap; import org.apache.druid.server.security.Authorizer; import org.apache.druid.server.security.AuthorizerMapper; @@ -69,6 +71,8 @@ public class CoordinatorPollingBasicAuthorizerCacheManager implements BasicAutho private final ConcurrentHashMap> cachedUserMaps; private final ConcurrentHashMap> cachedRoleMaps; + private final ConcurrentHashMap> cachedGroupMappingMaps; + private final ConcurrentHashMap> cachedGroupMappingRoleMaps; private final Set authorizerPrefixes; private final Injector injector; @@ -92,6 +96,8 @@ public CoordinatorPollingBasicAuthorizerCacheManager( this.objectMapper = objectMapper; this.cachedUserMaps = new ConcurrentHashMap<>(); this.cachedRoleMaps = new ConcurrentHashMap<>(); + this.cachedGroupMappingMaps = new ConcurrentHashMap<>(); + this.cachedGroupMappingRoleMaps = new ConcurrentHashMap<>(); this.authorizerPrefixes = new HashSet<>(); this.druidLeaderClient = druidLeaderClient; } @@ -118,7 +124,7 @@ public void start() LOG.debug("Inserting random polling delay of [%s] ms", randomDelay); Thread.sleep(randomDelay); - LOG.debug("Scheduled cache poll is running"); + LOG.debug("Scheduled userMap cache poll is running"); for (String authorizerPrefix : authorizerPrefixes) { UserAndRoleMap userAndRoleMap = fetchUserAndRoleMapFromCoordinator(authorizerPrefix, false); if (userAndRoleMap != null) { @@ -126,7 +132,7 @@ public void start() cachedRoleMaps.put(authorizerPrefix, userAndRoleMap.getRoleMap()); } } - LOG.debug("Scheduled cache poll is done"); + LOG.debug("Scheduled userMap cache poll is done"); } catch (Throwable t) { LOG.makeAlert(t, "Error occured while polling for cachedUserMaps.").emit(); @@ -134,6 +140,32 @@ public void start() } ); + ScheduledExecutors.scheduleWithFixedDelay( + exec, + new Duration(commonCacheConfig.getPollingPeriod()), + new Duration(commonCacheConfig.getPollingPeriod()), + () -> { + try { + long randomDelay = ThreadLocalRandom.current().nextLong(0, commonCacheConfig.getMaxRandomDelay()); + LOG.debug("Inserting random polling delay of [%s] ms", randomDelay); + Thread.sleep(randomDelay); + + LOG.debug("Scheduled groupMappingMap cache poll is running"); + for (String authorizerPrefix : authorizerPrefixes) { + GroupMappingAndRoleMap groupMappingAndRoleMap = fetchGroupAndRoleMapFromCoordinator(authorizerPrefix, false); + if (groupMappingAndRoleMap != null) { + cachedGroupMappingMaps.put(authorizerPrefix, groupMappingAndRoleMap.getGroupMappingMap()); + cachedGroupMappingRoleMaps.put(authorizerPrefix, groupMappingAndRoleMap.getRoleMap()); + } + } + LOG.debug("Scheduled groupMappingMap cache poll is done"); + } + catch (Throwable t) { + LOG.makeAlert(t, "Error occured while polling for cachedGroupMappingMaps.").emit(); + } + } + ); + lifecycleLock.started(); LOG.info("Started CoordinatorPollingBasicAuthorizerCacheManager."); } @@ -155,9 +187,9 @@ public void stop() } @Override - public void handleAuthorizerUpdate(String authorizerPrefix, byte[] serializedUserAndRoleMap) + public void handleAuthorizerUserUpdate(String authorizerPrefix, byte[] serializedUserAndRoleMap) { - LOG.debug("Received cache update for authorizer [%s].", authorizerPrefix); + LOG.debug("Received userMap cache update for authorizer [%s].", authorizerPrefix); Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); try { UserAndRoleMap userAndRoleMap = objectMapper.readValue( @@ -169,7 +201,7 @@ public void handleAuthorizerUpdate(String authorizerPrefix, byte[] serializedUse cachedRoleMaps.put(authorizerPrefix, userAndRoleMap.getRoleMap()); if (commonCacheConfig.getCacheDirectory() != null) { - writeMapToDisk(authorizerPrefix, serializedUserAndRoleMap); + writeUserMapToDisk(authorizerPrefix, serializedUserAndRoleMap); } } catch (Exception e) { @@ -177,6 +209,29 @@ public void handleAuthorizerUpdate(String authorizerPrefix, byte[] serializedUse } } + @Override + public void handleAuthorizerGroupMappingUpdate(String authorizerPrefix, byte[] serializedGroupMappingAndRoleMap) + { + LOG.debug("Received groupMappingMap cache update for authorizer [%s].", authorizerPrefix); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + try { + GroupMappingAndRoleMap groupMappingAndRoleMap = objectMapper.readValue( + serializedGroupMappingAndRoleMap, + BasicAuthUtils.AUTHORIZER_GROUP_MAPPING_AND_ROLE_MAP_TYPE_REFERENCE + ); + + cachedGroupMappingMaps.put(authorizerPrefix, groupMappingAndRoleMap.getGroupMappingMap()); + cachedGroupMappingRoleMaps.put(authorizerPrefix, groupMappingAndRoleMap.getRoleMap()); + + if (commonCacheConfig.getCacheDirectory() != null) { + writeGroupMappingMapToDisk(authorizerPrefix, serializedGroupMappingAndRoleMap); + } + } + catch (Exception e) { + LOG.makeAlert(e, "Could not deserialize groupMapping/role map received from coordinator.").emit(); + } + } + @Override public Map getUserMap(String authorizerPrefix) { @@ -189,9 +244,26 @@ public Map getRoleMap(String authorizerPrefix) return cachedRoleMaps.get(authorizerPrefix); } + @Override + public Map getGroupMappingMap(String authorizerPrefix) + { + return cachedGroupMappingMaps.get(authorizerPrefix); + } + + @Override + public Map getGroupMappingRoleMap(String authorizerPrefix) + { + return cachedGroupMappingRoleMaps.get(authorizerPrefix); + } + private String getUserRoleMapFilename(String prefix) { - return StringUtils.format("%s.authorizer.cache", prefix); + return StringUtils.format("%s.authorizer.userRole.cache", prefix); + } + + private String getGroupMappingRoleMapFilename(String prefix) + { + return StringUtils.format("%s.authorizer.groupMappingRole.cache", prefix); } @Nullable @@ -207,7 +279,20 @@ private UserAndRoleMap loadUserAndRoleMapFromDisk(String prefix) throws IOExcept ); } - private void writeMapToDisk(String prefix, byte[] userMapBytes) throws IOException + @Nullable + private GroupMappingAndRoleMap loadGroupMappingAndRoleMapFromDisk(String prefix) throws IOException + { + File groupMappingAndRoleMapFile = new File(commonCacheConfig.getCacheDirectory(), getGroupMappingRoleMapFilename(prefix)); + if (!groupMappingAndRoleMapFile.exists()) { + return null; + } + return objectMapper.readValue( + groupMappingAndRoleMapFile, + BasicAuthUtils.AUTHORIZER_GROUP_MAPPING_AND_ROLE_MAP_TYPE_REFERENCE + ); + } + + private void writeUserMapToDisk(String prefix, byte[] userMapBytes) throws IOException { File cacheDir = new File(commonCacheConfig.getCacheDirectory()); cacheDir.mkdirs(); @@ -221,13 +306,27 @@ private void writeMapToDisk(String prefix, byte[] userMapBytes) throws IOExcepti ); } + private void writeGroupMappingMapToDisk(String prefix, byte[] groupMappingBytes) throws IOException + { + File cacheDir = new File(commonCacheConfig.getCacheDirectory()); + cacheDir.mkdirs(); + File groupMapFile = new File(commonCacheConfig.getCacheDirectory(), getGroupMappingRoleMapFilename(prefix)); + FileUtils.writeAtomically( + groupMapFile, + out -> { + out.write(groupMappingBytes); + return null; + } + ); + } + @Nullable private UserAndRoleMap fetchUserAndRoleMapFromCoordinator(String prefix, boolean isInit) { try { return RetryUtils.retry( () -> { - return tryFetchMapsFromCoordinator(prefix); + return tryFetchUserMapsFromCoordinator(prefix); }, e -> true, commonCacheConfig.getMaxSyncRetries() @@ -252,7 +351,38 @@ private UserAndRoleMap fetchUserAndRoleMapFromCoordinator(String prefix, boolean } } - private UserAndRoleMap tryFetchMapsFromCoordinator( + @Nullable + private GroupMappingAndRoleMap fetchGroupAndRoleMapFromCoordinator(String prefix, boolean isInit) + { + try { + return RetryUtils.retry( + () -> { + return tryFetchGroupMappingMapsFromCoordinator(prefix); + }, + e -> true, + commonCacheConfig.getMaxSyncRetries() + ); + } + catch (Exception e) { + LOG.makeAlert(e, "Encountered exception while fetching group and role map for authorizer [%s]", prefix).emit(); + if (isInit) { + if (commonCacheConfig.getCacheDirectory() != null) { + try { + LOG.info("Attempting to load group map snapshot from disk."); + return loadGroupMappingAndRoleMapFromDisk(prefix); + } + catch (Exception e2) { + e2.addSuppressed(e); + LOG.makeAlert(e2, "Encountered exception while loading group-role map snapshot for authorizer [%s]", prefix) + .emit(); + } + } + } + return null; + } + } + + private UserAndRoleMap tryFetchUserMapsFromCoordinator( String prefix ) throws Exception { @@ -271,11 +401,35 @@ private UserAndRoleMap tryFetchMapsFromCoordinator( BasicAuthUtils.AUTHORIZER_USER_AND_ROLE_MAP_TYPE_REFERENCE ); if (userAndRoleMap != null && commonCacheConfig.getCacheDirectory() != null) { - writeMapToDisk(prefix, userRoleMapBytes); + writeUserMapToDisk(prefix, userRoleMapBytes); } return userAndRoleMap; } + private GroupMappingAndRoleMap tryFetchGroupMappingMapsFromCoordinator( + String prefix + ) throws Exception + { + Request req = druidLeaderClient.makeRequest( + HttpMethod.GET, + StringUtils.format("/druid-ext/basic-security/authorization/db/%s/cachedSerializedGroupMappingMap", prefix) + ); + BytesFullResponseHolder responseHolder = (BytesFullResponseHolder) druidLeaderClient.go( + req, + new BytesFullResponseHandler() + ); + byte[] groupRoleMapBytes = responseHolder.getBytes(); + + GroupMappingAndRoleMap groupMappingAndRoleMap = objectMapper.readValue( + groupRoleMapBytes, + BasicAuthUtils.AUTHORIZER_GROUP_MAPPING_AND_ROLE_MAP_TYPE_REFERENCE + ); + if (groupMappingAndRoleMap != null && commonCacheConfig.getCacheDirectory() != null) { + writeGroupMappingMapToDisk(prefix, groupRoleMapBytes); + } + return groupMappingAndRoleMap; + } + private void initUserMaps() { AuthorizerMapper authorizerMapper = injector.getInstance(AuthorizerMapper.class); @@ -289,11 +443,18 @@ private void initUserMaps() if (authorizer instanceof BasicRoleBasedAuthorizer) { String authorizerName = entry.getKey(); authorizerPrefixes.add(authorizerName); + UserAndRoleMap userAndRoleMap = fetchUserAndRoleMapFromCoordinator(authorizerName, true); if (userAndRoleMap != null) { cachedUserMaps.put(authorizerName, userAndRoleMap.getUserMap()); cachedRoleMaps.put(authorizerName, userAndRoleMap.getRoleMap()); } + + GroupMappingAndRoleMap groupMappingAndRoleMap = fetchGroupAndRoleMapFromCoordinator(authorizerName, true); + if (groupMappingAndRoleMap != null) { + cachedGroupMappingMaps.put(authorizerName, groupMappingAndRoleMap.getGroupMappingMap()); + cachedGroupMappingRoleMaps.put(authorizerName, groupMappingAndRoleMap.getRoleMap()); + } } } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/MetadataStoragePollingBasicAuthorizerCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/MetadataStoragePollingBasicAuthorizerCacheManager.java index 2c2c38587077..075b40ecadd1 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/MetadataStoragePollingBasicAuthorizerCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/MetadataStoragePollingBasicAuthorizerCacheManager.java @@ -21,6 +21,7 @@ import com.google.inject.Inject; import org.apache.druid.security.basic.authorization.db.updater.BasicAuthorizerMetadataStorageUpdater; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; @@ -39,7 +40,13 @@ public MetadataStoragePollingBasicAuthorizerCacheManager( } @Override - public void handleAuthorizerUpdate(String authorizerPrefix, byte[] serializedUserAndRoleMap) + public void handleAuthorizerUserUpdate(String authorizerPrefix, byte[] serializedUserAndRoleMap) + { + + } + + @Override + public void handleAuthorizerGroupMappingUpdate(String authorizerPrefix, byte[] serializedGroupMappingAndRoleMap) { } @@ -55,4 +62,16 @@ public Map getRoleMap(String authorizerPrefix) { return storageUpdater.getCachedRoleMap(authorizerPrefix); } + + @Override + public Map getGroupMappingMap(String authorizerPrefix) + { + return storageUpdater.getCachedGroupMappingMap(authorizerPrefix); + } + + @Override + public Map getGroupMappingRoleMap(String authorizerPrefix) + { + return getRoleMap(authorizerPrefix); + } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/BasicAuthorizerMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/BasicAuthorizerMetadataStorageUpdater.java index 0cfb394f12eb..e47830bc6c0f 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/BasicAuthorizerMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/BasicAuthorizerMetadataStorageUpdater.java @@ -19,6 +19,7 @@ package org.apache.druid.security.basic.authorization.db.updater; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; import org.apache.druid.server.security.ResourceAction; @@ -38,22 +39,34 @@ public interface BasicAuthorizerMetadataStorageUpdater void deleteUser(String prefix, String userName); + void createGroupMapping(String prefix, BasicAuthorizerGroupMapping groupMapping); + + void deleteGroupMapping(String prefix, String groupMappingName); + void createRole(String prefix, String roleName); void deleteRole(String prefix, String roleName); - void assignRole(String prefix, String userName, String roleName); + void assignUserRole(String prefix, String userName, String roleName); + + void unassignUserRole(String prefix, String userName, String roleName); - void unassignRole(String prefix, String userName, String roleName); + void assignGroupMappingRole(String prefix, String groupMappingName, String roleName); + + void unassignGroupMappingRole(String prefix, String groupMappingName, String roleName); void setPermissions(String prefix, String roleName, List permissions); Map getCachedUserMap(String prefix); + Map getCachedGroupMappingMap(String prefix); + Map getCachedRoleMap(String prefix); byte[] getCurrentUserMapBytes(String prefix); + byte[] getCurrentGroupMappingMapBytes(String prefix); + byte[] getCurrentRoleMapBytes(String prefix); void refreshAllNotification(); diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java index 5c80ec7d9ba9..7a0af069fd1c 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/updater/CoordinatorBasicAuthorizerMetadataStorageUpdater.java @@ -38,15 +38,19 @@ import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.security.basic.BasicAuthCommonCacheConfig; +import org.apache.druid.security.basic.BasicAuthDBConfig; import org.apache.druid.security.basic.BasicAuthUtils; import org.apache.druid.security.basic.BasicSecurityDBResourceException; import org.apache.druid.security.basic.authorization.BasicRoleBasedAuthorizer; import org.apache.druid.security.basic.authorization.db.cache.BasicAuthorizerCacheNotifier; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMappingMapBundle; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerPermission; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRoleMapBundle; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUserMapBundle; +import org.apache.druid.security.basic.authorization.entity.GroupMappingAndRoleMap; import org.apache.druid.security.basic.authorization.entity.UserAndRoleMap; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Authorizer; @@ -56,14 +60,15 @@ import org.apache.druid.server.security.ResourceType; import org.joda.time.Duration; +import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadLocalRandom; @@ -78,6 +83,7 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdater implements BasicAu private static final long UPDATE_RETRY_DELAY = 1000; private static final String USERS = "users"; + private static final String GROUP_MAPPINGS = "groupMappings"; private static final String ROLES = "roles"; public static final List SUPERUSER_PERMISSIONS = makeSuperUserPermissions(); @@ -91,6 +97,7 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdater implements BasicAu private final int numRetries = 5; private final Map cachedUserMaps; + private final Map cachedGroupMappingMaps; private final Map cachedRoleMaps; private final Set authorizerNames; @@ -118,6 +125,7 @@ public CoordinatorBasicAuthorizerMetadataStorageUpdater( this.objectMapper = objectMapper; this.cacheNotifier = cacheNotifier; this.cachedUserMaps = new ConcurrentHashMap<>(); + this.cachedGroupMappingMaps = new ConcurrentHashMap<>(); this.cachedRoleMaps = new ConcurrentHashMap<>(); this.authorizerNames = new HashSet<>(); } @@ -138,6 +146,8 @@ public void start() for (Map.Entry entry : authorizerMapper.getAuthorizerMap().entrySet()) { Authorizer authorizer = entry.getValue(); if (authorizer instanceof BasicRoleBasedAuthorizer) { + BasicRoleBasedAuthorizer basicRoleBasedAuthorizer = (BasicRoleBasedAuthorizer) authorizer; + BasicAuthDBConfig dbConfig = basicRoleBasedAuthorizer.getDbConfig(); String authorizerName = entry.getKey(); authorizerNames.add(authorizerName); @@ -148,6 +158,13 @@ public void start() ); cachedUserMaps.put(authorizerName, new BasicAuthorizerUserMapBundle(userMap, userMapBytes)); + byte[] groupMappingMapBytes = getCurrentGroupMappingMapBytes(authorizerName); + Map groupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + groupMappingMapBytes + ); + cachedGroupMappingMaps.put(authorizerName, new BasicAuthorizerGroupMappingMapBundle(groupMappingMap, groupMappingMapBytes)); + byte[] roleMapBytes = getCurrentRoleMapBytes(authorizerName); Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( objectMapper, @@ -155,7 +172,11 @@ public void start() ); cachedRoleMaps.put(authorizerName, new BasicAuthorizerRoleMapBundle(roleMap, roleMapBytes)); - initSuperusers(authorizerName, userMap, roleMap); + initSuperUsersAndGroupMapping(authorizerName, userMap, roleMap, groupMappingMap, + dbConfig.getInitialAdminUser(), + dbConfig.getInitialAdminRole(), + dbConfig.getInitialAdminGroupMapping() + ); } } @@ -163,47 +184,53 @@ public void start() exec, new Duration(commonCacheConfig.getPollingPeriod()), new Duration(commonCacheConfig.getPollingPeriod()), - new Callable() - { - @Override - public ScheduledExecutors.Signal call() - { - if (stopped) { - return ScheduledExecutors.Signal.STOP; - } - try { - LOG.debug("Scheduled db poll is running"); - for (String authorizerName : authorizerNames) { - - byte[] userMapBytes = getCurrentUserMapBytes(authorizerName); - Map userMap = BasicAuthUtils.deserializeAuthorizerUserMap( - objectMapper, - userMapBytes - ); - if (userMapBytes != null) { - synchronized (cachedUserMaps) { - cachedUserMaps.put(authorizerName, new BasicAuthorizerUserMapBundle(userMap, userMapBytes)); - } + () -> { + if (stopped) { + return ScheduledExecutors.Signal.STOP; + } + try { + LOG.debug("Scheduled db poll is running"); + for (String authorizerName : authorizerNames) { + + byte[] userMapBytes = getCurrentUserMapBytes(authorizerName); + Map userMap = BasicAuthUtils.deserializeAuthorizerUserMap( + objectMapper, + userMapBytes + ); + if (userMapBytes != null) { + synchronized (cachedUserMaps) { + cachedUserMaps.put(authorizerName, new BasicAuthorizerUserMapBundle(userMap, userMapBytes)); } + } - byte[] roleMapBytes = getCurrentRoleMapBytes(authorizerName); - Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( - objectMapper, - roleMapBytes - ); - if (roleMapBytes != null) { - synchronized (cachedUserMaps) { - cachedRoleMaps.put(authorizerName, new BasicAuthorizerRoleMapBundle(roleMap, roleMapBytes)); - } + byte[] groupMappingMapBytes = getCurrentGroupMappingMapBytes(authorizerName); + Map groupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + groupMappingMapBytes + ); + if (groupMappingMapBytes != null) { + synchronized (cachedGroupMappingMaps) { + cachedGroupMappingMaps.put(authorizerName, new BasicAuthorizerGroupMappingMapBundle(groupMappingMap, groupMappingMapBytes)); + } + } + + byte[] roleMapBytes = getCurrentRoleMapBytes(authorizerName); + Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( + objectMapper, + roleMapBytes + ); + if (roleMapBytes != null) { + synchronized (cachedRoleMaps) { + cachedRoleMaps.put(authorizerName, new BasicAuthorizerRoleMapBundle(roleMap, roleMapBytes)); } } - LOG.debug("Scheduled db poll is done"); - } - catch (Throwable t) { - LOG.makeAlert(t, "Error occured while polling for cachedUserMaps.").emit(); } - return ScheduledExecutors.Signal.REPEAT; + LOG.debug("Scheduled db poll is done"); + } + catch (Throwable t) { + LOG.makeAlert(t, "Error occured while polling for cachedUserMaps, cachedGroupMappingMaps, cachedRoleMaps.").emit(); } + return ScheduledExecutors.Signal.REPEAT; } ); @@ -239,7 +266,69 @@ private boolean tryUpdateUserMap( byte[] newUserMapValue ) { - return tryUpdateUserAndRoleMap(prefix, userMap, oldUserMapValue, newUserMapValue, null, null, null); + try { + List updates = new ArrayList<>(); + if (userMap != null) { + updates.add( + createMetadataCASUpdate(prefix, oldUserMapValue, newUserMapValue, USERS) + ); + + boolean succeeded = connector.compareAndSwap(updates); + if (succeeded) { + cachedUserMaps.put(prefix, new BasicAuthorizerUserMapBundle(userMap, newUserMapValue)); + + byte[] serializedUserAndRoleMap = getCurrentUserAndRoleMapSerialized(prefix); + cacheNotifier.addUpdateUser(prefix, serializedUserAndRoleMap); + + return true; + } else { + return false; + } + } + return false; + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private boolean tryUpdateGroupMappingMap( + String prefix, + Map groupMappingMap, + byte[] oldGroupMappingMapValue, + byte[] newGroupMappingMapValue + ) + { + try { + List updates = new ArrayList<>(); + if (groupMappingMap != null) { + updates.add( + createMetadataCASUpdate(prefix, oldGroupMappingMapValue, newGroupMappingMapValue, GROUP_MAPPINGS) + ); + + + boolean succeeded = connector.compareAndSwap(updates); + if (succeeded) { + cachedGroupMappingMaps.put(prefix, + new BasicAuthorizerGroupMappingMapBundle( + groupMappingMap, + newGroupMappingMapValue + ) + ); + + byte[] serializedGroupMappingAndRoleMap = getCurrentGroupMappingAndRoleMapSerialized(prefix); + cacheNotifier.addUpdateGroupMapping(prefix, serializedGroupMappingAndRoleMap); + + return true; + } else { + return false; + } + } + return false; + } + catch (Exception e) { + throw new RuntimeException(e); + } } private boolean tryUpdateRoleMap( @@ -249,7 +338,33 @@ private boolean tryUpdateRoleMap( byte[] newRoleMapValue ) { - return tryUpdateUserAndRoleMap(prefix, null, null, null, roleMap, oldRoleMapValue, newRoleMapValue); + try { + List updates = new ArrayList<>(); + if (roleMap != null) { + updates.add( + createMetadataCASUpdate(prefix, oldRoleMapValue, newRoleMapValue, ROLES) + ); + + boolean succeeded = connector.compareAndSwap(updates); + if (succeeded) { + + cachedRoleMaps.put(prefix, new BasicAuthorizerRoleMapBundle(roleMap, newRoleMapValue)); + + byte[] serializedUserAndRoleMap = getCurrentUserAndRoleMapSerialized(prefix); + cacheNotifier.addUpdateUser(prefix, serializedUserAndRoleMap); + byte[] serializedGroupMappingAndRoleMap = getCurrentGroupMappingAndRoleMapSerialized(prefix); + cacheNotifier.addUpdateGroupMapping(prefix, serializedGroupMappingAndRoleMap); + + return true; + } else { + return false; + } + } + return false; + } + catch (Exception e) { + throw new RuntimeException(e); + } } private boolean tryUpdateUserAndRoleMap( @@ -264,55 +379,91 @@ private boolean tryUpdateUserAndRoleMap( { try { List updates = new ArrayList<>(); - if (userMap != null) { + if (userMap != null && roleMap != null) { updates.add( - new MetadataCASUpdate( - connectorConfig.getConfigTable(), - MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, - MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, - getPrefixedKeyColumn(prefix, USERS), - oldUserMapValue, - newUserMapValue - ) + createMetadataCASUpdate(prefix, oldUserMapValue, newUserMapValue, USERS) ); + updates.add( + createMetadataCASUpdate(prefix, oldRoleMapValue, newRoleMapValue, ROLES) + ); + + boolean succeeded = connector.compareAndSwap(updates); + if (succeeded) { + cachedUserMaps.put(prefix, new BasicAuthorizerUserMapBundle(userMap, newUserMapValue)); + cachedRoleMaps.put(prefix, new BasicAuthorizerRoleMapBundle(roleMap, newRoleMapValue)); + + byte[] serializedUserAndRoleMap = getCurrentUserAndRoleMapSerialized(prefix); + cacheNotifier.addUpdateUser(prefix, serializedUserAndRoleMap); + + return true; + } else { + return false; + } } + } + catch (Exception e) { + throw new RuntimeException(e); + } + return false; + } - if (roleMap != null) { + private boolean tryUpdateGroupMappingAndRoleMap( + String prefix, + Map groupMappingMap, + byte[] oldGroupMappingMapValue, + byte[] newGroupMappingMapValue, + Map roleMap, + byte[] oldRoleMapValue, + byte[] newRoleMapValue + ) + { + try { + List updates = new ArrayList<>(); + if (groupMappingMap != null && roleMap != null) { updates.add( - new MetadataCASUpdate( - connectorConfig.getConfigTable(), - MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, - MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, - getPrefixedKeyColumn(prefix, ROLES), - oldRoleMapValue, - newRoleMapValue - ) + createMetadataCASUpdate(prefix, oldGroupMappingMapValue, newGroupMappingMapValue, GROUP_MAPPINGS) + ); + updates.add( + createMetadataCASUpdate(prefix, oldRoleMapValue, newRoleMapValue, ROLES) ); } boolean succeeded = connector.compareAndSwap(updates); if (succeeded) { - if (userMap != null) { - cachedUserMaps.put(prefix, new BasicAuthorizerUserMapBundle(userMap, newUserMapValue)); - } - if (roleMap != null) { - cachedRoleMaps.put(prefix, new BasicAuthorizerRoleMapBundle(roleMap, newRoleMapValue)); - } + cachedGroupMappingMaps.put(prefix, new BasicAuthorizerGroupMappingMapBundle(groupMappingMap, newGroupMappingMapValue)); + cachedRoleMaps.put(prefix, new BasicAuthorizerRoleMapBundle(roleMap, newRoleMapValue)); - byte[] serializedUserAndRoleMap = getCurrentUserAndRoleMapSerialized(prefix); - cacheNotifier.addUpdate(prefix, serializedUserAndRoleMap); + byte[] serializedGroupMappingAndRoleMap = getCurrentGroupMappingAndRoleMapSerialized(prefix); + cacheNotifier.addUpdateGroupMapping(prefix, serializedGroupMappingAndRoleMap); return true; } else { return false; } - } catch (Exception e) { throw new RuntimeException(e); } } + @Nonnull + private MetadataCASUpdate createMetadataCASUpdate( + String prefix, + byte[] oldValue, + byte[] newValue, + String columnName + ) + { + return new MetadataCASUpdate( + connectorConfig.getConfigTable(), + MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, + MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, + getPrefixedKeyColumn(prefix, columnName), + oldValue, + newValue + ); + } + @Override public void createUser(String prefix, String userName) { @@ -327,6 +478,21 @@ public void deleteUser(String prefix, String userName) deleteUserInternal(prefix, userName); } + @Override + public void createGroupMapping(String prefix, BasicAuthorizerGroupMapping groupMapping) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + createGroupMappingInternal(prefix, groupMapping); + + } + + @Override + public void deleteGroupMapping(String prefix, String groupMappingName) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + deleteGroupMappingInternal(prefix, groupMappingName); + } + @Override public void createRole(String prefix, String roleName) { @@ -342,17 +508,31 @@ public void deleteRole(String prefix, String roleName) } @Override - public void assignRole(String prefix, String userName, String roleName) + public void assignUserRole(String prefix, String userName, String roleName) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + assignUserRoleInternal(prefix, userName, roleName); + } + + @Override + public void unassignUserRole(String prefix, String userName, String roleName) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + unassignUserRoleInternal(prefix, userName, roleName); + } + + @Override + public void assignGroupMappingRole(String prefix, String groupMappingName, String roleName) { Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - assignRoleInternal(prefix, userName, roleName); + assignGroupMappingRoleInternal(prefix, groupMappingName, roleName); } @Override - public void unassignRole(String prefix, String userName, String roleName) + public void unassignGroupMappingRole(String prefix, String groupMappingName, String roleName) { Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - unassignRoleInternal(prefix, userName, roleName); + unassignGroupMappingRoleInternal(prefix, groupMappingName, roleName); } @Override @@ -370,6 +550,13 @@ public Map getCachedUserMap(String prefix) return userMapBundle == null ? null : userMapBundle.getUserMap(); } + @Override + public Map getCachedGroupMappingMap(String prefix) + { + BasicAuthorizerGroupMappingMapBundle groupMapBundle = cachedGroupMappingMaps.get(prefix); + return groupMapBundle == null ? null : groupMapBundle.getGroupMappingMap(); + } + @Override @Nullable public Map getCachedRoleMap(String prefix) @@ -389,6 +576,17 @@ public byte[] getCurrentUserMapBytes(String prefix) ); } + @Override + public byte[] getCurrentGroupMappingMapBytes(String prefix) + { + return connector.lookup( + connectorConfig.getConfigTable(), + MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, + MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, + getPrefixedKeyColumn(prefix, GROUP_MAPPINGS) + ); + } + @Override public byte[] getCurrentRoleMapBytes(String prefix) { @@ -407,7 +605,10 @@ public void refreshAllNotification() (authorizerName) -> { try { byte[] serializedUserAndRoleMap = getCurrentUserAndRoleMapSerialized(authorizerName); - cacheNotifier.addUpdate(authorizerName, serializedUserAndRoleMap); + cacheNotifier.addUpdateUser(authorizerName, serializedUserAndRoleMap); + + byte[] serializeGroupAndRoleMap = getCurrentGroupMappingAndRoleMapSerialized(authorizerName); + cacheNotifier.addUpdateGroupMapping(authorizerName, serializeGroupAndRoleMap); } catch (IOException ioe) { throw new RuntimeException(ioe); @@ -429,6 +630,19 @@ private byte[] getCurrentUserAndRoleMapSerialized(String prefix) throws IOExcept return objectMapper.writeValueAsBytes(userAndRoleMap); } + private byte[] getCurrentGroupMappingAndRoleMapSerialized(String prefix) throws IOException + { + BasicAuthorizerGroupMappingMapBundle groupMappingMapBundle = cachedGroupMappingMaps.get(prefix); + BasicAuthorizerRoleMapBundle roleMapBundle = cachedRoleMaps.get(prefix); + + GroupMappingAndRoleMap groupMappingAndRoleMap = new GroupMappingAndRoleMap( + groupMappingMapBundle == null ? null : groupMappingMapBundle.getGroupMappingMap(), + roleMapBundle == null ? null : roleMapBundle.getRoleMap() + ); + + return objectMapper.writeValueAsBytes(groupMappingAndRoleMap); + } + private void createUserInternal(String prefix, String userName) { int attempts = 0; @@ -445,7 +659,7 @@ private void createUserInternal(String prefix, String userName) throw new RuntimeException(ie); } } - throw new ISE("Could not create user[%s] due to concurrent update contention.", userName); + throw new ISE("Could not create user [%s] due to concurrent update contention.", userName); } private void deleteUserInternal(String prefix, String userName) @@ -464,7 +678,45 @@ private void deleteUserInternal(String prefix, String userName) throw new RuntimeException(ie); } } - throw new ISE("Could not delete user[%s] due to concurrent update contention.", userName); + throw new ISE("Could not delete user [%s] due to concurrent update contention.", userName); + } + + private void createGroupMappingInternal(String prefix, BasicAuthorizerGroupMapping groupMapping) + { + int attempts = 0; + while (attempts < numRetries) { + if (createGroupMappingOnce(prefix, groupMapping)) { + return; + } else { + attempts++; + } + try { + Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); + } + catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + throw new ISE("Could not create group mapping [%s] due to concurrent update contention.", groupMapping); + } + + private void deleteGroupMappingInternal(String prefix, String groupMappingName) + { + int attempts = 0; + while (attempts < numRetries) { + if (deleteGroupMappingOnce(prefix, groupMappingName)) { + return; + } else { + attempts++; + } + try { + Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); + } + catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + throw new ISE("Could not delete group mapping [%s] due to concurrent update contention.", groupMappingName); } private void createRoleInternal(String prefix, String roleName) @@ -483,7 +735,7 @@ private void createRoleInternal(String prefix, String roleName) throw new RuntimeException(ie); } } - throw new ISE("Could not create role[%s] due to concurrent update contention.", roleName); + throw new ISE("Could not create role [%s] due to concurrent update contention.", roleName); } private void deleteRoleInternal(String prefix, String roleName) @@ -502,14 +754,52 @@ private void deleteRoleInternal(String prefix, String roleName) throw new RuntimeException(ie); } } - throw new ISE("Could not delete role[%s] due to concurrent update contention.", roleName); + throw new ISE("Could not delete role [%s] due to concurrent update contention.", roleName); + } + + private void assignUserRoleInternal(String prefix, String userName, String roleName) + { + int attempts = 0; + while (attempts < numRetries) { + if (assignUserRoleOnce(prefix, userName, roleName)) { + return; + } else { + attempts++; + } + try { + Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); + } + catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + throw new ISE("Could not assign role [%s] to user [%s] due to concurrent update contention.", roleName, userName); + } + + private void unassignUserRoleInternal(String prefix, String userName, String roleName) + { + int attempts = 0; + while (attempts < numRetries) { + if (unassignUserRoleOnce(prefix, userName, roleName)) { + return; + } else { + attempts++; + } + try { + Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); + } + catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + throw new ISE("Could not unassign role [%s] from user [%s] due to concurrent update contention.", roleName, userName); } - private void assignRoleInternal(String prefix, String userName, String roleName) + private void assignGroupMappingRoleInternal(String prefix, String groupMappingName, String roleName) { int attempts = 0; while (attempts < numRetries) { - if (assignRoleOnce(prefix, userName, roleName)) { + if (assignGroupMappingRoleOnce(prefix, groupMappingName, roleName)) { return; } else { attempts++; @@ -521,14 +811,17 @@ private void assignRoleInternal(String prefix, String userName, String roleName) throw new RuntimeException(ie); } } - throw new ISE("Could not assign role[%s] to user[%s] due to concurrent update contention.", roleName, userName); + throw new ISE("Could not assign role [%s] to group mapping [%s] due to concurrent update contention.", + roleName, + groupMappingName + ); } - private void unassignRoleInternal(String prefix, String userName, String roleName) + private void unassignGroupMappingRoleInternal(String prefix, String groupMappingName, String roleName) { int attempts = 0; while (attempts < numRetries) { - if (unassignRoleOnce(prefix, userName, roleName)) { + if (unassignGroupMappingRoleOnce(prefix, groupMappingName, roleName)) { return; } else { attempts++; @@ -540,7 +833,9 @@ private void unassignRoleInternal(String prefix, String userName, String roleNam throw new RuntimeException(ie); } } - throw new ISE("Could not unassign role[%s] from user[%s] due to concurrent update contention.", roleName, userName); + throw new ISE("Could not unassign role [%s] from group mapping [%s] due to concurrent update contention.", roleName, + groupMappingName + ); } private void setPermissionsInternal(String prefix, String roleName, List permissions) @@ -559,35 +854,61 @@ private void setPermissionsInternal(String prefix, String roleName, List userMap = BasicAuthUtils.deserializeAuthorizerUserMap(objectMapper, oldValue); - if (userMap.get(userName) != null) { - throw new BasicSecurityDBResourceException("User [%s] already exists.", userName); + if (userMap.get(userName) == null) { + throw new BasicSecurityDBResourceException("User [%s] does not exist.", userName); } else { - userMap.put(userName, new BasicAuthorizerUser(userName, null)); + userMap.remove(userName); } byte[] newValue = BasicAuthUtils.serializeAuthorizerUserMap(objectMapper, userMap); return tryUpdateUserMap(prefix, userMap, oldValue, newValue); } - private boolean deleteUserOnce(String prefix, String userName) + private boolean createUserOnce(String prefix, String userName) { byte[] oldValue = getCurrentUserMapBytes(prefix); Map userMap = BasicAuthUtils.deserializeAuthorizerUserMap(objectMapper, oldValue); - if (userMap.get(userName) == null) { - throw new BasicSecurityDBResourceException("User [%s] does not exist.", userName); + if (userMap.get(userName) != null) { + throw new BasicSecurityDBResourceException("User [%s] already exists.", userName); } else { - userMap.remove(userName); + userMap.put(userName, new BasicAuthorizerUser(userName, null)); } byte[] newValue = BasicAuthUtils.serializeAuthorizerUserMap(objectMapper, userMap); return tryUpdateUserMap(prefix, userMap, oldValue, newValue); } + private boolean deleteGroupMappingOnce(String prefix, String groupMappingName) + { + byte[] oldValue = getCurrentGroupMappingMapBytes(prefix); + Map groupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap(objectMapper, oldValue); + if (groupMappingMap.get(groupMappingName) == null) { + throw new BasicSecurityDBResourceException("Group mapping [%s] does not exist.", groupMappingName); + } else { + groupMappingMap.remove(groupMappingName); + } + byte[] newValue = BasicAuthUtils.serializeAuthorizerGroupMappingMap(objectMapper, groupMappingMap); + return tryUpdateGroupMappingMap(prefix, groupMappingMap, oldValue, newValue); + } + + private boolean createGroupMappingOnce(String prefix, BasicAuthorizerGroupMapping groupMapping) + { + byte[] oldValue = getCurrentGroupMappingMapBytes(prefix); + Map groupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap(objectMapper, oldValue); + if (groupMappingMap.get(groupMapping.getName()) != null) { + throw new BasicSecurityDBResourceException("Group mapping [%s] already exists.", groupMapping.getName()); + } else { + groupMappingMap.put(groupMapping.getName(), groupMapping); + } + byte[] newValue = BasicAuthUtils.serializeAuthorizerGroupMappingMap(objectMapper, groupMappingMap); + return tryUpdateGroupMappingMap(prefix, groupMappingMap, oldValue, newValue); + } + private boolean createRoleOnce(String prefix, String roleName) { byte[] oldValue = getCurrentRoleMapBytes(prefix); @@ -623,16 +944,31 @@ private boolean deleteRoleOnce(String prefix, String roleName) user.getRoles().remove(roleName); } byte[] newUserMapValue = BasicAuthUtils.serializeAuthorizerUserMap(objectMapper, userMap); + + byte[] oldGroupMapValue = getCurrentGroupMappingMapBytes(prefix); + Map groupMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + oldGroupMapValue + ); + for (BasicAuthorizerGroupMapping group : groupMap.values()) { + group.getRoles().remove(roleName); + } + byte[] newGroupMapValue = BasicAuthUtils.serializeAuthorizerGroupMappingMap(objectMapper, groupMap); + byte[] newRoleMapValue = BasicAuthUtils.serializeAuthorizerRoleMap(objectMapper, roleMap); return tryUpdateUserAndRoleMap( prefix, userMap, oldUserMapValue, newUserMapValue, roleMap, oldRoleMapValue, newRoleMapValue + ) && tryUpdateGroupMappingAndRoleMap( + prefix, + groupMap, oldGroupMapValue, newGroupMapValue, + roleMap, newRoleMapValue, newRoleMapValue ); } - private boolean assignRoleOnce(String prefix, String userName, String roleName) + private boolean assignUserRoleOnce(String prefix, String userName, String roleName) { byte[] oldRoleMapValue = getCurrentRoleMapBytes(prefix); Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( @@ -668,7 +1004,7 @@ private boolean assignRoleOnce(String prefix, String userName, String roleName) ); } - private boolean unassignRoleOnce(String prefix, String userName, String roleName) + private boolean unassignUserRoleOnce(String prefix, String userName, String roleName) { byte[] oldRoleMapValue = getCurrentRoleMapBytes(prefix); Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( @@ -704,6 +1040,78 @@ private boolean unassignRoleOnce(String prefix, String userName, String roleName ); } + private boolean assignGroupMappingRoleOnce(String prefix, String groupMappingName, String roleName) + { + byte[] oldRoleMapValue = getCurrentRoleMapBytes(prefix); + Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( + objectMapper, + oldRoleMapValue + ); + if (roleMap.get(roleName) == null) { + throw new BasicSecurityDBResourceException("Role [%s] does not exist.", roleName); + } + + byte[] oldGroupMappingMapValue = getCurrentGroupMappingMapBytes(prefix); + Map groupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + oldGroupMappingMapValue + ); + BasicAuthorizerGroupMapping groupMapping = groupMappingMap.get(groupMappingName); + if (groupMappingMap.get(groupMappingName) == null) { + throw new BasicSecurityDBResourceException("Group mapping [%s] does not exist.", groupMappingName); + } + + if (groupMapping.getRoles().contains(roleName)) { + throw new BasicSecurityDBResourceException("Group mapping [%s] already has role [%s].", groupMappingName, roleName); + } + + groupMapping.getRoles().add(roleName); + byte[] newGroupMapValue = BasicAuthUtils.serializeAuthorizerGroupMappingMap(objectMapper, groupMappingMap); + + // Role map is unchanged, but submit as an update to ensure that the table didn't change (e.g., role deleted) + return tryUpdateGroupMappingAndRoleMap( + prefix, + groupMappingMap, oldGroupMappingMapValue, newGroupMapValue, + roleMap, oldRoleMapValue, oldRoleMapValue + ); + } + + private boolean unassignGroupMappingRoleOnce(String prefix, String groupMappingName, String roleName) + { + byte[] oldRoleMapValue = getCurrentRoleMapBytes(prefix); + Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( + objectMapper, + oldRoleMapValue + ); + if (roleMap.get(roleName) == null) { + throw new BasicSecurityDBResourceException("Role [%s] does not exist.", roleName); + } + + byte[] oldGroupMappingMapValue = getCurrentGroupMappingMapBytes(prefix); + Map groupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + oldGroupMappingMapValue + ); + BasicAuthorizerGroupMapping groupMapping = groupMappingMap.get(groupMappingName); + if (groupMappingMap.get(groupMappingName) == null) { + throw new BasicSecurityDBResourceException("Group mapping [%s] does not exist.", groupMappingName); + } + + if (!groupMapping.getRoles().contains(roleName)) { + throw new BasicSecurityDBResourceException("Group mapping [%s] does not have role [%s].", groupMappingName, roleName); + } + + groupMapping.getRoles().remove(roleName); + byte[] newGroupMapValue = BasicAuthUtils.serializeAuthorizerGroupMappingMap(objectMapper, groupMappingMap); + + // Role map is unchanged, but submit as an update to ensure that the table didn't change (e.g., role deleted) + return tryUpdateGroupMappingAndRoleMap( + prefix, + groupMappingMap, oldGroupMappingMapValue, newGroupMapValue, + roleMap, oldRoleMapValue, oldRoleMapValue + ); + } + private boolean setPermissionsOnce(String prefix, String roleName, List permissions) { byte[] oldRoleMapValue = getCurrentRoleMapBytes(prefix); @@ -723,10 +1131,14 @@ private boolean setPermissionsOnce(String prefix, String roleName, List userMap, - Map roleMap + Map roleMap, + Map groupMappingMap, + String initialAdminUser, + String initialAdminRole, + String initialAdminGroupMapping ) { if (!roleMap.containsKey(BasicAuthUtils.ADMIN_NAME)) { @@ -739,15 +1151,41 @@ private void initSuperusers( setPermissionsInternal(authorizerName, BasicAuthUtils.INTERNAL_USER_NAME, SUPERUSER_PERMISSIONS); } + if (!userMap.containsKey(BasicAuthUtils.ADMIN_NAME)) { + createUserInternal(authorizerName, BasicAuthUtils.ADMIN_NAME); + assignUserRoleInternal(authorizerName, BasicAuthUtils.ADMIN_NAME, BasicAuthUtils.ADMIN_NAME); + } + if (!userMap.containsKey(BasicAuthUtils.INTERNAL_USER_NAME)) { createUserInternal(authorizerName, BasicAuthUtils.INTERNAL_USER_NAME); - assignRoleInternal(authorizerName, BasicAuthUtils.INTERNAL_USER_NAME, BasicAuthUtils.INTERNAL_USER_NAME); + assignUserRoleInternal(authorizerName, BasicAuthUtils.INTERNAL_USER_NAME, BasicAuthUtils.INTERNAL_USER_NAME); } - if (!userMap.containsKey(BasicAuthUtils.ADMIN_NAME)) { - createUserInternal(authorizerName, BasicAuthUtils.ADMIN_NAME); - assignRoleInternal(authorizerName, BasicAuthUtils.ADMIN_NAME, BasicAuthUtils.ADMIN_NAME); + if (initialAdminRole != null + && !(initialAdminRole.equals(BasicAuthUtils.ADMIN_NAME) || initialAdminRole.equals(BasicAuthUtils.INTERNAL_USER_NAME)) + && !roleMap.containsKey(initialAdminRole)) { + createRoleInternal(authorizerName, initialAdminRole); + setPermissionsInternal(authorizerName, initialAdminRole, SUPERUSER_PERMISSIONS); } + + if (initialAdminUser != null + && !(initialAdminUser.equals(BasicAuthUtils.ADMIN_NAME) || initialAdminUser.equals(BasicAuthUtils.INTERNAL_USER_NAME)) + && !userMap.containsKey(initialAdminUser)) { + createUserInternal(authorizerName, initialAdminUser); + assignUserRoleInternal(authorizerName, initialAdminUser, initialAdminRole == null ? BasicAuthUtils.ADMIN_NAME : initialAdminRole); + } + + if (initialAdminGroupMapping != null && !groupMappingMap.containsKey(BasicAuthUtils.ADMIN_GROUP_MAPPING_NAME)) { + BasicAuthorizerGroupMapping groupMapping = + new BasicAuthorizerGroupMapping( + BasicAuthUtils.ADMIN_GROUP_MAPPING_NAME, + initialAdminGroupMapping, + new HashSet<>(Collections.singletonList(initialAdminRole == null ? BasicAuthUtils.ADMIN_NAME : initialAdminRole)) + ); + createGroupMappingInternal(authorizerName, groupMapping); + } + + } private static List makeSuperUserPermissions() diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResource.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResource.java index eadd72a07a50..6e1bc20bcb4d 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResource.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResource.java @@ -24,6 +24,7 @@ import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.guice.LazySingleton; import org.apache.druid.security.basic.BasicSecurityResourceFilter; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.server.security.ResourceAction; import javax.servlet.http.HttpServletRequest; @@ -108,6 +109,24 @@ public Response getAllUsers( return resourceHandler.getAllUsers(authorizerName); } + /** + * @param req HTTP request + * + * @return List of all groupMappings + */ + @GET + @Path("/db/{authorizerName}/groupMappings") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response getAllGroupMappings( + @Context HttpServletRequest req, + @PathParam("authorizerName") final String authorizerName + ) + { + return resourceHandler.getAllGroupMappings(authorizerName); + } + /** * @param req HTTP request * @param userName Name of user to retrieve information about @@ -129,6 +148,27 @@ public Response getUser( return resourceHandler.getUser(authorizerName, userName, full != null); } + /** + * @param req HTTP request + * @param groupMappingName Name of groupMapping to retrieve information about + * + * @return Name, groupPattern, roles, and permissions of the groupMapping with groupMappingName, 400 error response if groupMapping doesn't exist + */ + @GET + @Path("/db/{authorizerName}/groupMappings/{groupMappingName}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response getGroupMapping( + @Context HttpServletRequest req, + @PathParam("authorizerName") final String authorizerName, + @PathParam("groupMappingName") final String groupMappingName, + @QueryParam("full") String full + ) + { + return resourceHandler.getGroupMapping(authorizerName, groupMappingName, full != null); + } + /** * Create a new user with name userName * @@ -173,6 +213,54 @@ public Response deleteUser( return resourceHandler.deleteUser(authorizerName, userName); } + /** + * Create a new groupMapping with name groupMappingName + * + * @param req HTTP request + * @param groupMappingName Name to assign the new groupMapping + * + * @return OK response, or 400 error response if groupMapping already exists + */ + @POST + @Path("/db/{authorizerName}/groupMappings/{groupMappingName}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response createGroupMapping( + @Context HttpServletRequest req, + @PathParam("authorizerName") final String authorizerName, + @PathParam("groupMappingName") String groupMappingName, + BasicAuthorizerGroupMapping groupMapping + ) + { + return resourceHandler.createGroupMapping( + authorizerName, + new BasicAuthorizerGroupMapping(groupMappingName, groupMapping.getGroupPattern(), groupMapping.getRoles()) + ); + } + + /** + * Delete a groupMapping with name groupMappingName + * + * @param req HTTP request + * @param groupMappingName Name of groupMapping to delete + * + * @return OK response, or 400 error response if groupMapping doesn't exist + */ + @DELETE + @Path("/db/{authorizerName}/groupMappings/{groupMappingName}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response deleteGroupMapping( + @Context HttpServletRequest req, + @PathParam("authorizerName") final String authorizerName, + @PathParam("groupMappingName") String groupMappingName + ) + { + return resourceHandler.deleteGroupMapping(authorizerName, groupMappingName); + } + /** * @param req HTTP request * @@ -197,7 +285,7 @@ public Response getAllRoles( * @param req HTTP request * @param roleName Name of role * - * @return Role name, users with role, and permissions of role. 400 error if role doesn't exist. + * @return Role name, users with role, groupMappings with role, and permissions of role. 400 error if role doesn't exist. */ @GET @Path("/db/{authorizerName}/roles/{roleName}") @@ -306,6 +394,54 @@ public Response unassignRoleFromUser( return resourceHandler.unassignRoleFromUser(authorizerName, userName, roleName); } + /** + * Assign a role to a groupMapping. + * + * @param req HTTP request + * @param groupMappingName Name of groupMapping + * @param roleName Name of role + * + * @return OK response. 400 error if groupMapping/role don't exist, or if groupMapping already has the role + */ + @POST + @Path("/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response assignRoleToGroupMapping( + @Context HttpServletRequest req, + @PathParam("authorizerName") final String authorizerName, + @PathParam("groupMappingName") String groupMappingName, + @PathParam("roleName") String roleName + ) + { + return resourceHandler.assignRoleToGroupMapping(authorizerName, groupMappingName, roleName); + } + + /** + * Remove a role from a groupMapping. + * + * @param req HTTP request + * @param groupMappingName Name of groupMapping + * @param roleName Name of role + * + * @return OK response. 400 error if groupMapping/role don't exist, or if groupMapping does not have the role. + */ + @DELETE + @Path("/db/{authorizerName}/groupMappings/{groupMappingName}/roles/{roleName}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response unassignRoleFromGroupMapping( + @Context HttpServletRequest req, + @PathParam("authorizerName") final String authorizerName, + @PathParam("groupMappingName") String groupMappingName, + @PathParam("roleName") String roleName + ) + { + return resourceHandler.unassignRoleFromGroupMapping(authorizerName, groupMappingName, roleName); + } + /** * Set the permissions of a role. This replaces the previous permissions of the role. * @@ -330,6 +466,28 @@ public Response setRolePermissions( return resourceHandler.setRolePermissions(authorizerName, roleName, permissions); } + /** + * Get the permissions of a role. + * + * @param req HTTP request + * @param roleName Name of role + * + * @return OK response. 400 error if role doesn't exist. + */ + @GET + @Path("/db/{authorizerName}/roles/{roleName}/permissions") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response getRolePermissions( + @Context HttpServletRequest req, + @PathParam("authorizerName") final String authorizerName, + @PathParam("roleName") String roleName + ) + { + return resourceHandler.getRolePermissions(authorizerName, roleName); + } + /** * @param req HTTP request * @@ -345,24 +503,59 @@ public Response getCachedSerializedUserMap( @PathParam("authorizerName") final String authorizerName ) { - return resourceHandler.getCachedMaps(authorizerName); + return resourceHandler.getCachedUserMaps(authorizerName); + } + + /** + * @param req HTTP request + * + * @return serialized groupMapping map + */ + @GET + @Path("/db/{authorizerName}/cachedSerializedGroupMappingMap") + @Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response getCachedSerializedGroupMap( + @Context HttpServletRequest req, + @PathParam("authorizerName") final String authorizerName + ) + { + return resourceHandler.getCachedGroupMappingMaps(authorizerName); } /** - * Listen for update notifications for the auth storage + * Listen for update notifications for the user auth storage */ @POST - @Path("/listen/{authorizerName}") + @Path("/listen/users/{authorizerName}") @Produces(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(BasicSecurityResourceFilter.class) - public Response authorizerUpdateListener( + public Response authorizerUserUpdateListener( @Context HttpServletRequest req, @PathParam("authorizerName") final String authorizerName, byte[] serializedUserAndRoleMap ) { - return resourceHandler.authorizerUpdateListener(authorizerName, serializedUserAndRoleMap); + return resourceHandler.authorizerUserUpdateListener(authorizerName, serializedUserAndRoleMap); + } + + /** + * Listen for update notifications for the groupMapping auth storage + */ + @POST + @Path("/listen/groupMappings/{authorizerName}") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response authorizerGroupMappingUpdateListener( + @Context HttpServletRequest req, + @PathParam("authorizerName") final String authorizerName, + byte[] serializedGroupMappingAndRoleMap + ) + { + return resourceHandler.authorizerGroupMappingUpdateListener(authorizerName, serializedGroupMappingAndRoleMap); } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResourceHandler.java index ad69d7b6c11e..8f51288ecdd8 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResourceHandler.java @@ -19,6 +19,7 @@ package org.apache.druid.security.basic.authorization.endpoint; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.server.security.ResourceAction; import javax.ws.rs.core.Response; @@ -34,12 +35,20 @@ public interface BasicAuthorizerResourceHandler // coordinator methods Response getAllUsers(String authorizerName); + Response getAllGroupMappings(String authorizerName); + Response getUser(String authorizerName, String userName, boolean isFull); + Response getGroupMapping(String authorizerName, String groupMappingName, boolean isFull); + Response createUser(String authorizerName, String userName); + Response createGroupMapping(String authorizerName, BasicAuthorizerGroupMapping groupMapping); + Response deleteUser(String authorizerName, String userName); + Response deleteGroupMapping(String authorizerName, String groupMappingName); + Response getAllRoles(String authorizerName); Response getRole(String authorizerName, String roleName, boolean isFull); @@ -50,16 +59,26 @@ public interface BasicAuthorizerResourceHandler Response assignRoleToUser(String authorizerName, String userName, String roleName); + Response assignRoleToGroupMapping(String authorizerName, String groupMappingName, String roleName); + Response unassignRoleFromUser(String authorizerName, String userName, String roleName); + Response unassignRoleFromGroupMapping(String authorizerName, String groupMappingName, String roleName); + Response setRolePermissions(String authorizerName, String roleName, List permissions); - Response getCachedMaps(String authorizerName); + Response getRolePermissions(String authorizerName, String roleName); + + Response getCachedUserMaps(String authorizerName); + + Response getCachedGroupMappingMaps(String authorizerName); Response refreshAll(); // non-coordinator methods - Response authorizerUpdateListener(String authorizerName, byte[] serializedUserAndRoleMap); + Response authorizerUserUpdateListener(String authorizerName, byte[] serializedUserAndRoleMap); + + Response authorizerGroupMappingUpdateListener(String authorizerName, byte[] serializedGroupMappingAndRoleMap); // common Response getLoadStatus(); diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/CoordinatorBasicAuthorizerResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/CoordinatorBasicAuthorizerResourceHandler.java index 6bd951fccdc8..4acad40d3f4e 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/CoordinatorBasicAuthorizerResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/CoordinatorBasicAuthorizerResourceHandler.java @@ -29,10 +29,13 @@ import org.apache.druid.security.basic.BasicSecurityDBResourceException; import org.apache.druid.security.basic.authorization.BasicRoleBasedAuthorizer; import org.apache.druid.security.basic.authorization.db.updater.BasicAuthorizerMetadataStorageUpdater; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMappingFull; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRoleFull; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUserFull; +import org.apache.druid.security.basic.authorization.entity.GroupMappingAndRoleMap; import org.apache.druid.security.basic.authorization.entity.UserAndRoleMap; import org.apache.druid.server.security.Authorizer; import org.apache.druid.server.security.AuthorizerMapper; @@ -91,6 +94,21 @@ public Response getAllUsers(String authorizerName) return Response.ok(userMap.keySet()).build(); } + @Override + public Response getAllGroupMappings(String authorizerName) + { + final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); + if (authorizer == null) { + return makeResponseForAuthorizerNotFound(authorizerName); + } + + Map groupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + storageUpdater.getCurrentGroupMappingMapBytes(authorizerName) + ); + return Response.ok(groupMappingMap.keySet()).build(); + } + @Override public Response getUser(String authorizerName, String userName, boolean isFull) { @@ -106,6 +124,21 @@ public Response getUser(String authorizerName, String userName, boolean isFull) } } + @Override + public Response getGroupMapping(String authorizerName, String groupMappingName, boolean isFull) + { + final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); + if (authorizer == null) { + return makeResponseForAuthorizerNotFound(authorizerName); + } + + if (isFull) { + return getGroupMappingFull(authorizerName, groupMappingName); + } else { + return getGroupMappingSimple(authorizerName, groupMappingName); + } + } + @Override public Response createUser(String authorizerName, String userName) { @@ -123,6 +156,23 @@ public Response createUser(String authorizerName, String userName) } } + @Override + public Response createGroupMapping(String authorizerName, BasicAuthorizerGroupMapping groupMapping) + { + final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); + if (authorizer == null) { + return makeResponseForAuthorizerNotFound(authorizerName); + } + + try { + storageUpdater.createGroupMapping(authorizerName, groupMapping); + return Response.ok().build(); + } + catch (BasicSecurityDBResourceException cfe) { + return makeResponseForBasicSecurityDBResourceException(cfe); + } + } + @Override public Response deleteUser(String authorizerName, String userName) { @@ -140,6 +190,23 @@ public Response deleteUser(String authorizerName, String userName) } } + @Override + public Response deleteGroupMapping(String authorizerName, String groupMappingName) + { + final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); + if (authorizer == null) { + return makeResponseForAuthorizerNotFound(authorizerName); + } + + try { + storageUpdater.deleteGroupMapping(authorizerName, groupMappingName); + return Response.ok().build(); + } + catch (BasicSecurityDBResourceException cfe) { + return makeResponseForBasicSecurityDBResourceException(cfe); + } + } + @Override public Response getAllRoles(String authorizerName) { @@ -214,7 +281,24 @@ public Response assignRoleToUser(String authorizerName, String userName, String } try { - storageUpdater.assignRole(authorizerName, userName, roleName); + storageUpdater.assignUserRole(authorizerName, userName, roleName); + return Response.ok().build(); + } + catch (BasicSecurityDBResourceException cfe) { + return makeResponseForBasicSecurityDBResourceException(cfe); + } + } + + @Override + public Response assignRoleToGroupMapping(String authorizerName, String groupMappingName, String roleName) + { + final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); + if (authorizer == null) { + return makeResponseForAuthorizerNotFound(authorizerName); + } + + try { + storageUpdater.assignGroupMappingRole(authorizerName, groupMappingName, roleName); return Response.ok().build(); } catch (BasicSecurityDBResourceException cfe) { @@ -231,7 +315,24 @@ public Response unassignRoleFromUser(String authorizerName, String userName, Str } try { - storageUpdater.unassignRole(authorizerName, userName, roleName); + storageUpdater.unassignUserRole(authorizerName, userName, roleName); + return Response.ok().build(); + } + catch (BasicSecurityDBResourceException cfe) { + return makeResponseForBasicSecurityDBResourceException(cfe); + } + } + + @Override + public Response unassignRoleFromGroupMapping(String authorizerName, String groupMappingName, String roleName) + { + final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); + if (authorizer == null) { + return makeResponseForAuthorizerNotFound(authorizerName); + } + + try { + storageUpdater.unassignGroupMappingRole(authorizerName, groupMappingName, roleName); return Response.ok().build(); } catch (BasicSecurityDBResourceException cfe) { @@ -257,7 +358,18 @@ public Response setRolePermissions(String authorizerName, String roleName, List< } @Override - public Response getCachedMaps(String authorizerName) + public Response getRolePermissions(String authorizerName, String roleName) + { + final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); + if (authorizer == null) { + return makeResponseForAuthorizerNotFound(authorizerName); + } + + return getPermissions(authorizerName, roleName); + } + + @Override + public Response getCachedUserMaps(String authorizerName) { final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); if (authorizer == null) { @@ -272,6 +384,21 @@ public Response getCachedMaps(String authorizerName) return Response.ok(userAndRoleMap).build(); } + @Override + public Response getCachedGroupMappingMaps(String authorizerName) + { + final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); + if (authorizer == null) { + return makeResponseForAuthorizerNotFound(authorizerName); + } + GroupMappingAndRoleMap groupMappingAndRoleMap = new GroupMappingAndRoleMap( + storageUpdater.getCachedGroupMappingMap(authorizerName), + storageUpdater.getCachedRoleMap(authorizerName) + ); + + return Response.ok(groupMappingAndRoleMap).build(); + } + @Override public Response refreshAll() { @@ -280,7 +407,13 @@ public Response refreshAll() } @Override - public Response authorizerUpdateListener(String authorizerName, byte[] serializedUserAndRoleMap) + public Response authorizerUserUpdateListener(String authorizerName, byte[] serializedUserAndRoleMap) + { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + @Override + public Response authorizerGroupMappingUpdateListener(String authorizerName, byte[] serializedGroupMappingAndRoleMap) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -291,7 +424,9 @@ public Response getLoadStatus() Map loadStatus = new HashMap<>(); authorizerMap.forEach( (authorizerName, authorizer) -> { - loadStatus.put(authorizerName, storageUpdater.getCachedUserMap(authorizerName) != null); + loadStatus.put(authorizerName, storageUpdater.getCachedUserMap(authorizerName) != null && + storageUpdater.getCachedGroupMappingMap(authorizerName) != null && + storageUpdater.getCachedRoleMap(authorizerName) != null); } ); return Response.ok(loadStatus).build(); @@ -342,17 +477,17 @@ private Response getUserFull(String authorizerName, String userName) storageUpdater.getCurrentUserMapBytes(authorizerName) ); - Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( - objectMapper, - storageUpdater.getCurrentRoleMapBytes(authorizerName) - ); - try { BasicAuthorizerUser user = userMap.get(userName); if (user == null) { throw new BasicSecurityDBResourceException("User [%s] does not exist.", userName); } + Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( + objectMapper, + storageUpdater.getCurrentRoleMapBytes(authorizerName) + ); + Set roles = new HashSet<>(); for (String roleName : user.getRoles()) { BasicAuthorizerRole role = roleMap.get(roleName); @@ -371,6 +506,61 @@ private Response getUserFull(String authorizerName, String userName) } } + private Response getGroupMappingSimple(String authorizerName, String groupMappingName) + { + Map groupMappings = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + storageUpdater.getCurrentGroupMappingMapBytes(authorizerName) + ); + + try { + BasicAuthorizerGroupMapping groupMapping = groupMappings.get(groupMappingName); + if (groupMapping == null) { + throw new BasicSecurityDBResourceException("Group mapping [%s] does not exist.", groupMappingName); + } + return Response.ok(groupMapping).build(); + } + catch (BasicSecurityDBResourceException e) { + return makeResponseForBasicSecurityDBResourceException(e); + } + } + + private Response getGroupMappingFull(String authorizerName, String groupMappingName) + { + Map groupMappings = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + storageUpdater.getCurrentGroupMappingMapBytes(authorizerName) + ); + + try { + BasicAuthorizerGroupMapping groupMapping = groupMappings.get(groupMappingName); + if (groupMapping == null) { + throw new BasicSecurityDBResourceException("Group mapping [%s] does not exist.", groupMappingName); + } + + Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( + objectMapper, + storageUpdater.getCurrentRoleMapBytes(authorizerName) + ); + + Set roles = new HashSet<>(); + for (String roleName : groupMapping.getRoles()) { + BasicAuthorizerRole role = roleMap.get(roleName); + if (role == null) { + log.error("Group mapping [%s] had role [%s], but role was not found.", groupMappingName, roleName); + } else { + roles.add(role); + } + } + + BasicAuthorizerGroupMappingFull fullGroup = new BasicAuthorizerGroupMappingFull(groupMapping.getName(), groupMapping.getGroupPattern(), roles); + return Response.ok(fullGroup).build(); + } + catch (BasicSecurityDBResourceException e) { + return makeResponseForBasicSecurityDBResourceException(e); + } + } + private Response getRoleSimple(String authorizerName, String roleName) { Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( @@ -397,26 +587,40 @@ private Response getRoleFull(String authorizerName, String roleName) storageUpdater.getCurrentRoleMapBytes(authorizerName) ); - Map userMap = BasicAuthUtils.deserializeAuthorizerUserMap( - objectMapper, - storageUpdater.getCurrentUserMapBytes(authorizerName) - ); - - Set users = new HashSet<>(); - for (BasicAuthorizerUser user : userMap.values()) { - if (user.getRoles().contains(roleName)) { - users.add(user.getName()); - } - } - try { BasicAuthorizerRole role = roleMap.get(roleName); if (role == null) { throw new BasicSecurityDBResourceException("Role [%s] does not exist.", roleName); } + + Map userMap = BasicAuthUtils.deserializeAuthorizerUserMap( + objectMapper, + storageUpdater.getCurrentUserMapBytes(authorizerName) + ); + + Map groupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + storageUpdater.getCurrentGroupMappingMapBytes(authorizerName) + ); + + Set users = new HashSet<>(); + for (BasicAuthorizerUser user : userMap.values()) { + if (user.getRoles().contains(roleName)) { + users.add(user.getName()); + } + } + + Set groupMappings = new HashSet<>(); + for (BasicAuthorizerGroupMapping group : groupMappingMap.values()) { + if (group.getRoles().contains(roleName)) { + groupMappings.add(group.getName()); + } + } + BasicAuthorizerRoleFull roleFull = new BasicAuthorizerRoleFull( roleName, users, + groupMappings, role.getPermissions() ); return Response.ok(roleFull).build(); @@ -425,4 +629,23 @@ private Response getRoleFull(String authorizerName, String roleName) return makeResponseForBasicSecurityDBResourceException(e); } } + + private Response getPermissions(String authorizerName, String roleName) + { + Map roleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( + objectMapper, + storageUpdater.getCurrentRoleMapBytes(authorizerName) + ); + + try { + BasicAuthorizerRole role = roleMap.get(roleName); + if (role == null) { + throw new BasicSecurityDBResourceException("Role [%s] does not exist.", roleName); + } + return Response.ok(role.getPermissions()).build(); + } + catch (BasicSecurityDBResourceException e) { + return makeResponseForBasicSecurityDBResourceException(e); + } + } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/DefaultBasicAuthorizerResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/DefaultBasicAuthorizerResourceHandler.java index 94a63b6f85a5..55d2242a206a 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/DefaultBasicAuthorizerResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/DefaultBasicAuthorizerResourceHandler.java @@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.security.basic.authorization.BasicRoleBasedAuthorizer; import org.apache.druid.security.basic.authorization.db.cache.BasicAuthorizerCacheManager; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.server.security.Authorizer; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ResourceAction; @@ -70,24 +71,49 @@ public Response getAllUsers(String authorizerName) return NOT_FOUND_RESPONSE; } + @Override + public Response getAllGroupMappings(String authorizerName) + { + return NOT_FOUND_RESPONSE; + } + + @Override public Response getUser(String authorizerName, String userName, boolean isFull) { return NOT_FOUND_RESPONSE; } + @Override + public Response getGroupMapping(String authorizerName, String groupMappingName, boolean isFull) + { + return NOT_FOUND_RESPONSE; + } + @Override public Response createUser(String authorizerName, String userName) { return NOT_FOUND_RESPONSE; } + @Override + public Response createGroupMapping(String authorizerName, BasicAuthorizerGroupMapping groupMapping) + { + return NOT_FOUND_RESPONSE; + } + @Override public Response deleteUser(String authorizerName, String userName) { return NOT_FOUND_RESPONSE; } + @Override + public Response deleteGroupMapping(String authorizerName, String groupMappingName) + { + return NOT_FOUND_RESPONSE; + } + @Override public Response getAllRoles(String authorizerName) { @@ -118,12 +144,24 @@ public Response assignRoleToUser(String authorizerName, String userName, String return NOT_FOUND_RESPONSE; } + @Override + public Response assignRoleToGroupMapping(String authorizerName, String groupMappingName, String roleName) + { + return NOT_FOUND_RESPONSE; + } + @Override public Response unassignRoleFromUser(String authorizerName, String userName, String roleName) { return NOT_FOUND_RESPONSE; } + @Override + public Response unassignRoleFromGroupMapping(String authorizerName, String groupMappingName, String roleName) + { + return NOT_FOUND_RESPONSE; + } + @Override public Response setRolePermissions(String authorizerName, String roleName, List permissions) { @@ -131,7 +169,19 @@ public Response setRolePermissions(String authorizerName, String roleName, List< } @Override - public Response getCachedMaps(String authorizerName) + public Response getRolePermissions(String authorizerName, String roleName) + { + return NOT_FOUND_RESPONSE; + } + + @Override + public Response getCachedUserMaps(String authorizerName) + { + return NOT_FOUND_RESPONSE; + } + + @Override + public Response getCachedGroupMappingMaps(String authorizerName) { return NOT_FOUND_RESPONSE; } @@ -143,7 +193,26 @@ public Response refreshAll() } @Override - public Response authorizerUpdateListener(String authorizerName, byte[] serializedUserAndRoleMap) + public Response authorizerUserUpdateListener(String authorizerName, byte[] serializedUserAndRoleMap) + { + final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); + if (authorizer == null) { + String errMsg = StringUtils.format("Received update for unknown authorizer[%s]", authorizerName); + log.error(errMsg); + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of( + "error", + StringUtils.format(errMsg) + )) + .build(); + } + + cacheManager.handleAuthorizerUserUpdate(authorizerName, serializedUserAndRoleMap); + return Response.ok().build(); + } + + @Override + public Response authorizerGroupMappingUpdateListener(String authorizerName, byte[] serializedGroupMappingAndRoleMap) { final BasicRoleBasedAuthorizer authorizer = authorizerMap.get(authorizerName); if (authorizer == null) { @@ -157,7 +226,7 @@ public Response authorizerUpdateListener(String authorizerName, byte[] serialize .build(); } - cacheManager.handleAuthorizerUpdate(authorizerName, serializedUserAndRoleMap); + cacheManager.handleAuthorizerGroupMappingUpdate(authorizerName, serializedGroupMappingAndRoleMap); return Response.ok().build(); } @@ -167,7 +236,10 @@ public Response getLoadStatus() Map loadStatus = new HashMap<>(); authorizerMap.forEach( (authorizerName, authorizer) -> { - loadStatus.put(authorizerName, cacheManager.getUserMap(authorizerName) != null); + loadStatus.put(authorizerName, cacheManager.getUserMap(authorizerName) != null && + cacheManager.getRoleMap(authorizerName) != null && + cacheManager.getGroupMappingMap(authorizerName) != null && + cacheManager.getGroupMappingRoleMap(authorizerName) != null); } ); return Response.ok(loadStatus).build(); diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMapping.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMapping.java new file mode 100644 index 000000000000..9c036ce6005b --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMapping.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authorization.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.HashSet; +import java.util.Set; + +public class BasicAuthorizerGroupMapping +{ + private final String name; + private final String groupPattern; + private final Set roles; + + @JsonCreator + public BasicAuthorizerGroupMapping( + @JsonProperty("name") String name, + @JsonProperty("groupPattern") String groupPattern, + @JsonProperty("roles") Set roles + ) + { + this.name = name; + this.groupPattern = groupPattern; + this.roles = roles == null ? new HashSet<>() : roles; + } + + @JsonProperty + public String getName() + { + return name; + } + + @JsonProperty + public String getGroupPattern() + { + return groupPattern; + } + + @JsonProperty + public Set getRoles() + { + return roles; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + BasicAuthorizerGroupMapping that = (BasicAuthorizerGroupMapping) o; + + if (getName() != null ? !getName().equals(that.getName()) : that.getName() != null) { + return false; + } + + if (getGroupPattern() != null ? !getGroupPattern().equals(that.getGroupPattern()) : that.getGroupPattern() != null) { + return false; + } + + return getRoles() != null ? getRoles().equals(that.getRoles()) : that.getRoles() == null; + + } + + @Override + public int hashCode() + { + int result = getName() != null ? getName().hashCode() : 0; + result = 31 * result + + (getGroupPattern() != null ? getGroupPattern().hashCode() : 0) + + (getRoles() != null ? getRoles().hashCode() : 0); + return result; + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingFull.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingFull.java new file mode 100644 index 000000000000..459c9b273486 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingFull.java @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authorization.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.HashSet; +import java.util.Set; + +public class BasicAuthorizerGroupMappingFull +{ + private final String name; + private final String groupPattern; + private final Set roles; + + @JsonCreator + public BasicAuthorizerGroupMappingFull( + @JsonProperty("name") String name, + @JsonProperty("groupPattern") String groupPattern, + @JsonProperty("roles") Set roles + ) + { + this.name = name; + this.groupPattern = groupPattern; + this.roles = roles == null ? new HashSet<>() : roles; + } + + @JsonProperty + public String getName() + { + return name; + } + + public String getGroupPattern() + { + return groupPattern; + } + + @JsonProperty + public Set getRoles() + { + return roles; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + BasicAuthorizerGroupMappingFull that = (BasicAuthorizerGroupMappingFull) o; + + if (getName() != null ? !getName().equals(that.getName()) : that.getName() != null) { + return false; + } + + if (getGroupPattern() != null ? !getGroupPattern().equals(that.getGroupPattern()) : that.getGroupPattern() != null) { + return false; + } + + return getRoles() != null ? getRoles().equals(that.getRoles()) : that.getRoles() == null; + + } + + @Override + public int hashCode() + { + int result = getName() != null ? getName().hashCode() : 0; + result = 31 * result + + (getGroupPattern() != null ? getGroupPattern().hashCode() : 0) + + (getRoles() != null ? getRoles().hashCode() : 0); + return result; + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingMapBundle.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingMapBundle.java new file mode 100644 index 000000000000..71af4bab678d --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingMapBundle.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authorization.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; + +public class BasicAuthorizerGroupMappingMapBundle +{ + private final Map groupMappingMap; + private final byte[] serializedGroupMappingMap; + + @JsonCreator + public BasicAuthorizerGroupMappingMapBundle( + @JsonProperty("groupMappingMap") Map groupMappingMap, + @JsonProperty("serializedGroupMappingMap") byte[] serializedGroupMappingMap + ) + { + this.groupMappingMap = groupMappingMap; + this.serializedGroupMappingMap = serializedGroupMappingMap; + } + + @JsonProperty + public Map getGroupMappingMap() + { + return groupMappingMap; + } + + @JsonProperty + public byte[] getSerializedGroupMappingMap() + { + return serializedGroupMappingMap; + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerRoleFull.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerRoleFull.java index 79ced60bb186..5a167c39dada 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerRoleFull.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerRoleFull.java @@ -30,17 +30,20 @@ public class BasicAuthorizerRoleFull { private final String name; private final Set users; + private final Set groups; private final List permissions; @JsonCreator public BasicAuthorizerRoleFull( @JsonProperty("name") String name, @JsonProperty("users") Set users, + @JsonProperty("groups") Set groups, @JsonProperty("permissions") List permissions ) { this.name = name; this.users = users; + this.groups = groups; this.permissions = permissions == null ? new ArrayList<>() : permissions; } @@ -62,6 +65,12 @@ public Set getUsers() return users; } + @JsonProperty + public Set getGroups() + { + return groups; + } + @Override public boolean equals(Object o) { @@ -80,6 +89,9 @@ public boolean equals(Object o) if (getUsers() != null ? !getUsers().equals(that.getUsers()) : that.getUsers() != null) { return false; } + if (getGroups() != null ? !getGroups().equals(that.getGroups()) : that.getGroups() != null) { + return false; + } return getPermissions() != null ? getPermissions().equals(that.getPermissions()) : that.getPermissions() == null; } @@ -89,6 +101,7 @@ public int hashCode() { int result = getName() != null ? getName().hashCode() : 0; result = 31 * result + (getUsers() != null ? getUsers().hashCode() : 0); + result = 31 * result + (getGroups() != null ? getGroups().hashCode() : 0); result = 31 * result + (getPermissions() != null ? getPermissions().hashCode() : 0); return result; } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/GroupMappingAndRoleMap.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/GroupMappingAndRoleMap.java new file mode 100644 index 000000000000..1ac9f65190da --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/GroupMappingAndRoleMap.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.authorization.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Map; + +public class GroupMappingAndRoleMap +{ + @JsonProperty + private Map groupMappingMap; + + @JsonProperty + private Map roleMap; + + @JsonCreator + public GroupMappingAndRoleMap( + @JsonProperty("groupMappingMap") Map groupMappingMap, + @JsonProperty("roleMap") Map roleMap + ) + { + this.groupMappingMap = groupMappingMap; + this.roleMap = roleMap; + } + + @JsonProperty + public Map getGroupMappingMap() + { + return groupMappingMap; + } + + @JsonProperty + public Map getRoleMap() + { + return roleMap; + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheManager.java new file mode 100644 index 000000000000..c7fac703b12a --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheManager.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.db.cache; + +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; + +/** + * This class is reponsible for maintaining a cache of the escalator database state. The BasicHTTPEscalator + * uses an injected BasicEscalatorCacheManager to set escalator credentials. + */ +public interface BasicEscalatorCacheManager +{ + /** + * Update this cache manager's local state of escalator credential with fresh information pushed by the coordinator. + * + * @param serializedEscalatorCredentialConfig The updated, serialized escalator credential + */ + void handleEscalatorCredentialUpdate(byte[] serializedEscalatorCredentialConfig); + + /** + * Return the cache manager's local view of escalator credential. + * + * @return Escalator credential + */ + BasicEscalatorCredential getEscalatorCredential(); +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheNotifier.java new file mode 100644 index 000000000000..3c07a8877cab --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheNotifier.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.db.cache; + +/** + * Sends a notification to druid services, containing updated escalator state. + */ +public interface BasicEscalatorCacheNotifier +{ + /** + * Send the escalator credential state contained in updatedEscalatorCredential to all non-coordinator Druid services + * + * @param updatedEscalatorCredential Escalator credential state + */ + void addEscalatorCredentialUpdate(byte[] updatedEscalatorCredential); +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java new file mode 100644 index 000000000000..0eb68b8f1907 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.db.cache; + +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import org.apache.druid.concurrent.LifecycleLock; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.EscalatedClient; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.security.basic.BasicAuthDBConfig; +import org.apache.druid.security.basic.CommonCacheNotifier; +import org.apache.druid.security.basic.authentication.BasicHTTPEscalator; +import org.apache.druid.server.security.Escalator; + +import java.util.concurrent.TimeUnit; + +@ManageLifecycle +public class CoordinatorBasicEscalatorCacheNotifier implements BasicEscalatorCacheNotifier +{ + + private final LifecycleLock lifecycleLock = new LifecycleLock(); + private CommonCacheNotifier escalatorCredentialCacheNotifier; + + @Inject + public CoordinatorBasicEscalatorCacheNotifier( + DruidNodeDiscoveryProvider discoveryProvider, + final Escalator escalator, + @EscalatedClient HttpClient httpClient + ) + { + escalatorCredentialCacheNotifier = new CommonCacheNotifier( + initEscalatorConfigMap(escalator), + discoveryProvider, + httpClient, + "/druid-ext/basic-security/escalator/listen/credential", + "CoordinatorBasicEscalatorCacheNotifier" + ); + } + + @LifecycleStart + public void start() + { + if (!lifecycleLock.canStart()) { + throw new ISE("can't start."); + } + + try { + escalatorCredentialCacheNotifier.start(); + lifecycleLock.started(); + } + finally { + lifecycleLock.exitStart(); + } + } + + @LifecycleStop + public void stop() + { + if (!lifecycleLock.canStop()) { + return; + } + try { + escalatorCredentialCacheNotifier.stop(); + } + finally { + lifecycleLock.exitStop(); + } + } + + @Override + public void addEscalatorCredentialUpdate(byte[] updatedEscalatorCredential) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + escalatorCredentialCacheNotifier.addUpdate(updatedEscalatorCredential); + } + + private BasicAuthDBConfig initEscalatorConfigMap(Escalator escalator) + { + Preconditions.checkNotNull(escalator); + + BasicAuthDBConfig dbConfig; + if (escalator instanceof BasicHTTPEscalator) { + BasicHTTPEscalator basicHTTPEscalator = (BasicHTTPEscalator) escalator; + dbConfig = basicHTTPEscalator.getDbConfig(); + } else { + dbConfig = new BasicAuthDBConfig( + null, + null, + null, + null, + null, + true, + BasicAuthDBConfig.DEFAULT_CACHE_NOTIFY_TIMEOUT_MS, + 0, + null, + null, + null, + null, + null, + null, + null, null, + null, + null + ); + } + return dbConfig; + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java new file mode 100644 index 000000000000..66089a3b7183 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.db.cache; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import com.google.inject.Injector; +import org.apache.druid.client.coordinator.Coordinator; +import org.apache.druid.concurrent.LifecycleLock; +import org.apache.druid.discovery.DruidLeaderClient; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.RetryUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.security.basic.BasicAuthCommonCacheConfig; +import org.apache.druid.security.basic.authentication.BasicHTTPEscalator; +import org.apache.druid.security.basic.authentication.BytesFullResponseHandler; +import org.apache.druid.security.basic.authentication.BytesFullResponseHolder; +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; +import org.apache.druid.server.security.Escalator; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.joda.time.Duration; + +import javax.annotation.Nullable; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +/** + * Cache manager for non-coordinator services that polls the coordinator for escalator database state. + */ +@ManageLifecycle +public class CoordinatorPollingBasicEscalatorCacheManager implements BasicEscalatorCacheManager +{ + private static final EmittingLogger LOG = new EmittingLogger(CoordinatorPollingBasicEscalatorCacheManager.class); + + private final AtomicReference cachedEscalatorCredential; + private final Injector injector; + private final ObjectMapper objectMapper; + private final LifecycleLock lifecycleLock = new LifecycleLock(); + private final DruidLeaderClient druidLeaderClient; + private final BasicAuthCommonCacheConfig commonCacheConfig; + private final ScheduledExecutorService exec; + + @Inject + public CoordinatorPollingBasicEscalatorCacheManager( + Injector injector, + BasicAuthCommonCacheConfig commonCacheConfig, + @Smile ObjectMapper objectMapper, + @Coordinator DruidLeaderClient druidLeaderClient + ) + { + this.exec = Execs.scheduledSingleThreaded("BasicAuthenticatorCacheManager-Exec--%d"); + this.injector = injector; + this.commonCacheConfig = commonCacheConfig; + this.objectMapper = objectMapper; + this.druidLeaderClient = druidLeaderClient; + this.cachedEscalatorCredential = new AtomicReference<>(); + } + + @LifecycleStart + public void start() + { + if (!lifecycleLock.canStart()) { + throw new ISE("Can't start."); + } + + LOG.info("Starting CoordinatorPollingBasicEscalatorCacheManager."); + + try { + initEscalatorCredential(); + + ScheduledExecutors.scheduleWithFixedDelay( + exec, + new Duration(commonCacheConfig.getPollingPeriod()), + new Duration(commonCacheConfig.getPollingPeriod()), + () -> { + try { + long randomDelay = ThreadLocalRandom.current().nextLong(0, commonCacheConfig.getMaxRandomDelay()); + LOG.debug("Inserting cachedConfigs random polling delay of [%s] ms", randomDelay); + Thread.sleep(randomDelay); + + LOG.debug("Scheduled escalator credential cache poll is running"); + BasicEscalatorCredential escalatorCredential = fetchEscalatorCredentialFromCoordinator(); + if (escalatorCredential != null) { + cachedEscalatorCredential.set(escalatorCredential); + } + LOG.debug("Scheduled escalator credential cache poll is done"); + } + catch (Throwable t) { + LOG.makeAlert(t, "Error occured while polling for cachedEscalatorCredential.").emit(); + } + } + ); + + lifecycleLock.started(); + LOG.info("Started CoordinatorPollingBasicEscalatorCacheManager."); + } + finally { + lifecycleLock.exitStart(); + } + } + + @LifecycleStop + public void stop() + { + if (!lifecycleLock.canStop()) { + throw new ISE("can't stop."); + } + + LOG.info("CoordinatorPollingBasicEscalatorCacheManager is stopping."); + exec.shutdown(); + LOG.info("CoordinatorPollingBasicEscalatorCacheManager is stopped."); + } + + @Override + public void handleEscalatorCredentialUpdate(byte[] serializedEscalatorCredentialConfig) + { + LOG.debug("Received escalator credential cache update for escalator."); + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + try { + cachedEscalatorCredential.set( + objectMapper.readValue(serializedEscalatorCredentialConfig, BasicEscalatorCredential.class) + ); + } + catch (Exception e) { + LOG.makeAlert(e, "Could not deserialize escalator credential received from coordinator.").emit(); + } + } + + @Override + public BasicEscalatorCredential getEscalatorCredential() + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + + return cachedEscalatorCredential.get(); + } + + @Nullable + private BasicEscalatorCredential fetchEscalatorCredentialFromCoordinator() + { + try { + return RetryUtils.retry( + this::tryFetchEscalatorCredentialFromCoordinator, + e -> true, + commonCacheConfig.getMaxSyncRetries() + ); + } + catch (Exception e) { + LOG.makeAlert(e, "Encountered exception while fetching escalator credential for escalator").emit(); + return null; + } + } + + private BasicEscalatorCredential tryFetchEscalatorCredentialFromCoordinator() throws Exception + { + Request req = druidLeaderClient.makeRequest( + HttpMethod.GET, + "/druid-ext/basic-security/escalator/db/cachedSerializedCredential" + ); + BytesFullResponseHolder responseHolder = (BytesFullResponseHolder) druidLeaderClient.go( + req, + new BytesFullResponseHandler() + ); + byte[] escalatorCredentialBytes = responseHolder.getBytes(); + + return objectMapper.readValue(escalatorCredentialBytes, BasicEscalatorCredential.class); + } + + private void initEscalatorCredential() + { + Escalator escalator = injector.getInstance(Escalator.class); + + if (escalator == null) { + return; + } + + if (escalator instanceof BasicHTTPEscalator) { + BasicEscalatorCredential escalatorCredential = fetchEscalatorCredentialFromCoordinator(); + if (escalatorCredential != null) { + cachedEscalatorCredential.set(escalatorCredential); + } + } + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/MetadataStoragePollingBasicEscalatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/MetadataStoragePollingBasicEscalatorCacheManager.java new file mode 100644 index 000000000000..b5e656f99e5d --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/MetadataStoragePollingBasicEscalatorCacheManager.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.db.cache; + +import com.google.inject.Inject; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.security.basic.escalator.db.updater.BasicEscalatorMetadataStorageUpdater; +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; + +/** + * Used on coordinator nodes, reading from a BasicEscalatorMetadataStorageUpdater that has direct access to the + * metadata store. + */ +public class MetadataStoragePollingBasicEscalatorCacheManager implements BasicEscalatorCacheManager +{ + private static final Logger log = new Logger(MetadataStoragePollingBasicEscalatorCacheManager.class); + + private final BasicEscalatorMetadataStorageUpdater storageUpdater; + + @Inject + public MetadataStoragePollingBasicEscalatorCacheManager( + BasicEscalatorMetadataStorageUpdater storageUpdater + ) + { + this.storageUpdater = storageUpdater; + + log.info("Starting coordinator basic escalator cache manager."); + } + + @Override + public void handleEscalatorCredentialUpdate(byte[] serializedEscalatorCredentialConfig) + { + } + + @Override + public BasicEscalatorCredential getEscalatorCredential() + { + return storageUpdater.getCachedEscalatorCredential(); + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/BasicEscalatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/BasicEscalatorMetadataStorageUpdater.java new file mode 100644 index 000000000000..3289d80763e1 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/BasicEscalatorMetadataStorageUpdater.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.db.updater; + +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; + +/** + * Implementations of this interface are responsible for connecting directly to the metadata storage, + * modifying the escalator database state or reading it. This interface is used by the + * MetadataStoragePollingBasicEscalatorCacheManager (for reads) and the CoordinatorBasicEscalatorResourceHandler + * (for handling configuration read/writes). + */ +public interface BasicEscalatorMetadataStorageUpdater +{ + void updateEscalatorCredential(BasicEscalatorCredential escalatorCredential); + + BasicEscalatorCredential getCachedEscalatorCredential(); + + byte[] getCachedSerializedEscalatorCredential(); + + byte[] getCurrentEscalatorCredentialBytes(); + + void refreshAllNotification(); + +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java new file mode 100644 index 000000000000..34765bf0df77 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java @@ -0,0 +1,277 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.db.updater; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; +import com.google.inject.Inject; +import org.apache.druid.common.config.ConfigManager; +import org.apache.druid.concurrent.LifecycleLock; +import org.apache.druid.guice.ManageLifecycle; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; +import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.metadata.MetadataCASUpdate; +import org.apache.druid.metadata.MetadataStorageConnector; +import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.security.basic.BasicAuthCommonCacheConfig; +import org.apache.druid.security.basic.BasicAuthUtils; +import org.apache.druid.security.basic.escalator.db.cache.BasicEscalatorCacheNotifier; +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredentialBundle; +import org.joda.time.Duration; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + +@ManageLifecycle +public class CoordinatorBasicEscalatorMetadataStorageUpdater implements BasicEscalatorMetadataStorageUpdater +{ + private static final EmittingLogger LOG = new EmittingLogger(CoordinatorBasicEscalatorMetadataStorageUpdater.class); + + + private static final String CREDENTIAL = "credential"; + private static final long UPDATE_RETRY_DELAY = 1000; + + private final MetadataStorageConnector connector; + private final MetadataStorageTablesConfig connectorConfig; + private final BasicAuthCommonCacheConfig commonCacheConfig; + private final ObjectMapper objectMapper; + private final BasicEscalatorCacheNotifier cacheNotifier; + private final int numRetries = 5; + + private final AtomicReference cachedEscalatorCredential; + private final Set authenticatorPrefixes; + private final LifecycleLock lifecycleLock = new LifecycleLock(); + + private final ScheduledExecutorService exec; + private volatile boolean stopped = false; + + @Inject + public CoordinatorBasicEscalatorMetadataStorageUpdater( + MetadataStorageConnector connector, + MetadataStorageTablesConfig connectorConfig, + BasicAuthCommonCacheConfig commonCacheConfig, + @Smile ObjectMapper objectMapper, + BasicEscalatorCacheNotifier cacheNotifier, + ConfigManager configManager // ConfigManager creates the db table we need, set a dependency here + ) + { + this.exec = Execs.scheduledSingleThreaded("CoordinatorBasicEscalatorMetadataStorageUpdater-Exec--%d"); + this.connector = connector; + this.connectorConfig = connectorConfig; + this.commonCacheConfig = commonCacheConfig; + this.objectMapper = objectMapper; + this.cacheNotifier = cacheNotifier; + this.cachedEscalatorCredential = new AtomicReference<>(); + this.authenticatorPrefixes = new HashSet<>(); + } + + @LifecycleStart + public void start() + { + if (!lifecycleLock.canStart()) { + throw new ISE("can't start."); + } + + try { + LOG.info("Starting CoordinatorBasicEscalatorMetadataStorageUpdater."); + ScheduledExecutors.scheduleWithFixedDelay( + exec, + new Duration(commonCacheConfig.getPollingPeriod()), + new Duration(commonCacheConfig.getPollingPeriod()), + new Callable() + { + @Override + public ScheduledExecutors.Signal call() + { + if (stopped) { + return ScheduledExecutors.Signal.STOP; + } + try { + LOG.debug("Scheduled db escalator credential poll is running"); + byte[] escalatorCredentialBytes = getCurrentEscalatorCredentialBytes(); + + if (escalatorCredentialBytes != null) { + BasicEscalatorCredential escalatorCredential = BasicAuthUtils.deserializeEscalatorCredential( + objectMapper, + escalatorCredentialBytes + ); + if (escalatorCredential != null) { + cachedEscalatorCredential.set(new BasicEscalatorCredentialBundle(escalatorCredential, escalatorCredentialBytes)); + } + } + LOG.debug("Scheduled db escalator credential poll is done"); + } + catch (Throwable t) { + LOG.makeAlert(t, "Error occured while polling for cachedEscalatorCredential.").emit(); + } + return ScheduledExecutors.Signal.REPEAT; + } + } + ); + + lifecycleLock.started(); + } + finally { + lifecycleLock.exitStart(); + } + } + + @LifecycleStop + public void stop() + { + if (!lifecycleLock.canStop()) { + throw new ISE("can't stop."); + } + + LOG.info("CoordinatorBasicEscalatorMetadataStorageUpdater is stopping."); + stopped = true; + LOG.info("CoordinatorBasicEscalatorMetadataStorageUpdater is stopped."); + } + + + + @Override + public void refreshAllNotification() + { + cacheNotifier.addEscalatorCredentialUpdate(cachedEscalatorCredential.get().getSerializedEscalatorCredential()); + } + + @Override + public void updateEscalatorCredential(BasicEscalatorCredential escalatorCredential) + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + + updateEscalatorCredentialInternal(escalatorCredential); + } + + @Override + public BasicEscalatorCredential getCachedEscalatorCredential() + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + + BasicEscalatorCredentialBundle bundle = cachedEscalatorCredential.get(); + if (bundle == null) { + return null; + } else { + return bundle.getEscalatorCredential(); + } + } + + @Override + public byte[] getCachedSerializedEscalatorCredential() + { + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + + BasicEscalatorCredentialBundle bundle = cachedEscalatorCredential.get(); + if (bundle == null) { + return null; + } else { + return bundle.getSerializedEscalatorCredential(); + } + } + + @Override + public byte[] getCurrentEscalatorCredentialBytes() + { + return connector.lookup( + connectorConfig.getConfigTable(), + MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, + MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, + getKeyColumn(CREDENTIAL) + ); + } + + private static String getKeyColumn(String keyName) + { + return StringUtils.format("basic_escalator_%s", keyName); + } + + private void updateEscalatorCredentialInternal(BasicEscalatorCredential credential) + { + int attempts = 0; + while (attempts < numRetries) { + if (updateEscalatorCredentialOnce(credential)) { + return; + } else { + attempts++; + } + try { + Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); + } + catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + throw new ISE("Could not update escalator credential due to concurrent update contention."); + } + + private boolean updateEscalatorCredentialOnce(BasicEscalatorCredential credential) + { + byte[] oldValue = getCurrentEscalatorCredentialBytes(); + byte[] newValue = BasicAuthUtils.serializeEscalatorCredential(objectMapper, credential); + return tryUpdateEscalatorCredential(credential, oldValue, newValue); + } + + private boolean tryUpdateEscalatorCredential( + BasicEscalatorCredential credential, + byte[] oldValue, + byte[] newValue + ) + { + try { + MetadataCASUpdate update = new MetadataCASUpdate( + connectorConfig.getConfigTable(), + MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, + MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, + getKeyColumn(CREDENTIAL), + oldValue, + newValue + ); + + boolean succeeded = connector.compareAndSwap( + Collections.singletonList(update) + ); + + if (succeeded) { + cachedEscalatorCredential.set(new BasicEscalatorCredentialBundle(credential, newValue)); + cacheNotifier.addEscalatorCredentialUpdate(newValue); + return true; + } else { + return false; + } + } + catch (Exception e) { + throw new RuntimeException(e); + } + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResource.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResource.java new file mode 100644 index 000000000000..04e443688889 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResource.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.endpoint; + +import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.inject.Inject; +import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.druid.guice.LazySingleton; +import org.apache.druid.security.basic.BasicSecurityResourceFilter; +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; + +import javax.servlet.http.HttpServletRequest; +import javax.ws.rs.Consumes; +import javax.ws.rs.GET; +import javax.ws.rs.POST; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.Context; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; + +@Path("/druid-ext/basic-security/escalator") +@LazySingleton +public class BasicEscalatorResource +{ + private final BasicEscalatorResourceHandler handler; + + @Inject + public BasicEscalatorResource( + BasicEscalatorResourceHandler handler + ) + { + this.handler = handler; + } + + /** + * @param req HTTP request + * + * @return escalator credential + */ + @GET + @Path("/db/credential") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response getEscalatorCredential( + @Context HttpServletRequest req + ) + { + return handler.getEscalatorCredential(); + } + + /** + * Update escalator credential + * + * @param req HTTP request + * @param escalatorCredential Escalator credential + * + * @return OK response + */ + @POST + @Path("/db/credential") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response updateEscalatorCredential( + @Context HttpServletRequest req, + BasicEscalatorCredential escalatorCredential + ) + { + return handler.updateEscalatorCredential(escalatorCredential); + } + + /** + * @param req HTTP request + * + * @return serialized escalator credential + */ + @GET + @Path("/db/cachedSerializedCredential") + @Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response getCachedSerializedEscalatorCredential( + @Context HttpServletRequest req + ) + { + return handler.getCachedSerializedEscalatorCredential(); + } + + /** + * Listen for escalator credential update notifications for the escalator storage + */ + @POST + @Path("/listen/credential") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response escalatorCredentialUpdateListener( + @Context HttpServletRequest req, + byte[] serializedEscalatorCredential + ) + { + return handler.escalatorCredentialUpdateListener(serializedEscalatorCredential); + } + + /** + * @param req HTTP request + * + * @return Load status of escalator DB caches + */ + @GET + @Path("/loadStatus") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response getLoadStatus( + @Context HttpServletRequest req + ) + { + return handler.getLoadStatus(); + } + + /** + * @param req HTTP request + * + * Sends an "update" notification to all services with the current escalator database state, + * causing them to refresh their DB cache state. + */ + @GET + @Path("/refreshAll") + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + @ResourceFilters(BasicSecurityResourceFilter.class) + public Response refreshAll( + @Context HttpServletRequest req + ) + { + return handler.refreshAll(); + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResourceHandler.java new file mode 100644 index 000000000000..4dcae6cc91ef --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResourceHandler.java @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.endpoint; + +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; + +import javax.ws.rs.core.Response; + +/** + * Handles authenticator-related API calls. Coordinator and non-coordinator methods are combined here because of an + * inability to selectively inject jetty resources in configure(Binder binder) of the extension module based + * on node type. + */ +public interface BasicEscalatorResourceHandler +{ + // coordinator methods + Response getEscalatorCredential(); + + Response updateEscalatorCredential(BasicEscalatorCredential escalatorCredential); + + Response getCachedSerializedEscalatorCredential(); + + Response refreshAll(); + + // non-coordinator methods + Response escalatorCredentialUpdateListener(byte[] serializedEscalatorCredential); + + // common methods + Response getLoadStatus(); +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java new file mode 100644 index 000000000000..8898c5c2a506 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.endpoint; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import org.apache.druid.guice.annotations.Smile; +import org.apache.druid.security.basic.BasicAuthUtils; +import org.apache.druid.security.basic.BasicSecurityDBResourceException; +import org.apache.druid.security.basic.escalator.db.updater.BasicEscalatorMetadataStorageUpdater; +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; + +import javax.ws.rs.core.Response; + +public class CoordinatorBasicEscalatorResourceHandler implements BasicEscalatorResourceHandler +{ + private final BasicEscalatorMetadataStorageUpdater storageUpdater; + private final ObjectMapper objectMapper; + + @Inject + public CoordinatorBasicEscalatorResourceHandler( + BasicEscalatorMetadataStorageUpdater storageUpdater, + @Smile ObjectMapper objectMapper + ) + { + this.storageUpdater = storageUpdater; + this.objectMapper = objectMapper; + } + + @Override + public Response refreshAll() + { + storageUpdater.refreshAllNotification(); + return Response.ok().build(); + } + + @Override + public Response getEscalatorCredential() + { + BasicEscalatorCredential escalatorCredential = BasicAuthUtils.deserializeEscalatorCredential( + objectMapper, + storageUpdater.getCurrentEscalatorCredentialBytes() + ); + if (escalatorCredential == null) { + return Response.status(Response.Status.NOT_FOUND) + .entity(ImmutableMap.of( + "error", + "Escalator credential does not exist." + )) + .build(); + } else { + BasicEscalatorCredential maskedEscalatorCredential = new BasicEscalatorCredential( + escalatorCredential.getUsername(), + "..." + ); + + return Response.ok(maskedEscalatorCredential).build(); + } + } + + @Override + public Response updateEscalatorCredential(BasicEscalatorCredential escalatorCredential) + { + try { + storageUpdater.updateEscalatorCredential(escalatorCredential); + return Response.ok().build(); + } + catch (BasicSecurityDBResourceException cfe) { + return makeResponseForBasicSecurityDBResourceException(cfe); + } + } + + @Override + public Response getCachedSerializedEscalatorCredential() + { + return Response.ok(storageUpdater.getCachedSerializedEscalatorCredential()).build(); + } + + @Override + public Response escalatorCredentialUpdateListener(byte[] serializedEscalatorCredential) + { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + @Override + public Response getLoadStatus() + { + Boolean loadStatus = storageUpdater.getCachedEscalatorCredential() != null; + return Response.ok(loadStatus).build(); + } + + private static Response makeResponseForBasicSecurityDBResourceException(BasicSecurityDBResourceException bsre) + { + return Response.status(Response.Status.BAD_REQUEST) + .entity(ImmutableMap.of( + "error", bsre.getMessage() + )) + .build(); + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java new file mode 100644 index 000000000000..865826e9f161 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.endpoint; + +import com.google.inject.Inject; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.security.basic.escalator.db.cache.BasicEscalatorCacheManager; +import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; +import org.apache.druid.server.security.AuthenticatorMapper; + +import javax.ws.rs.core.Response; + +public class DefaultBasicEscalatorResourceHandler implements BasicEscalatorResourceHandler +{ + private static final Logger LOG = new Logger(DefaultBasicEscalatorResourceHandler.class); + private static final Response NOT_FOUND_RESPONSE = Response.status(Response.Status.NOT_FOUND).build(); + + private final BasicEscalatorCacheManager cacheManager; + + @Inject + public DefaultBasicEscalatorResourceHandler( + BasicEscalatorCacheManager cacheManager, + AuthenticatorMapper authenticatorMapper + ) + { + this.cacheManager = cacheManager; + } + + @Override + public Response refreshAll() + { + return NOT_FOUND_RESPONSE; + } + + @Override + public Response getEscalatorCredential() + { + return NOT_FOUND_RESPONSE; + } + + @Override + public Response updateEscalatorCredential(BasicEscalatorCredential escalatorCredential) + { + return NOT_FOUND_RESPONSE; + } + + @Override + public Response getCachedSerializedEscalatorCredential() + { + return NOT_FOUND_RESPONSE; + } + + @Override + public Response escalatorCredentialUpdateListener(byte[] serializedEscalatorCredential) + { + cacheManager.handleEscalatorCredentialUpdate(serializedEscalatorCredential); + return Response.ok().build(); + } + + @Override + public Response getLoadStatus() + { + Boolean loadStatus = cacheManager.getEscalatorCredential() != null; + return Response.ok(loadStatus).build(); + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java new file mode 100644 index 000000000000..14627e89cc20 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.entity; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class BasicEscalatorCredential +{ + private final String username; + private final String password; + + public BasicEscalatorCredential( + @JsonProperty("username") String username, + @JsonProperty("password") String password + ) + { + this.username = username; + this.password = password; + } + + @JsonProperty + public String getUsername() + { + return username; + } + + @JsonProperty + public String getPassword() + { + return password; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + BasicEscalatorCredential that = (BasicEscalatorCredential) o; + + if (getUsername() != null ? !getUsername().equals(that.getUsername()) : that.getUsername() != null) { + return false; + } + return getPassword() != null ? getPassword().equals(that.getPassword()) : that.getPassword() == null; + } + + @Override + public int hashCode() + { + int result = getUsername() != null ? getUsername().hashCode() : 0; + result = 31 * result + (getPassword() != null ? getPassword().hashCode() : 0); + return result; + } +} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredentialBundle.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredentialBundle.java new file mode 100644 index 000000000000..2108ff0d2b08 --- /dev/null +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredentialBundle.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.basic.escalator.entity; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class BasicEscalatorCredentialBundle +{ + private final BasicEscalatorCredential escalatorCredential; + private final byte[] serializedEscalatorCredential; + + @JsonCreator + public BasicEscalatorCredentialBundle( + @JsonProperty("escalatorCredential") BasicEscalatorCredential credential, + @JsonProperty("serializedEscalatorCredential") byte[] serializedEscalatorCredential + ) + { + this.escalatorCredential = credential; + this.serializedEscalatorCredential = serializedEscalatorCredential; + } + + @JsonProperty + public BasicEscalatorCredential getEscalatorCredential() + { + return escalatorCredential; + } + + @JsonProperty + public byte[] getSerializedEscalatorCredential() + { + return serializedEscalatorCredential; + } +} diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java index 07134973e09d..41de4a9baeab 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java @@ -24,11 +24,14 @@ import com.google.inject.util.Providers; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.metadata.DefaultPasswordProvider; +import org.apache.druid.security.basic.BasicSecurityAuthenticationException; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; +import org.apache.druid.security.basic.authentication.validator.CredentialsValidator; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthenticationResult; import org.easymock.EasyMock; @@ -52,7 +55,7 @@ public class BasicHTTPAuthenticatorTest new BasicAuthenticatorCacheManager() { @Override - public void handleAuthenticatorUpdate(String authenticatorPrefix, byte[] serializedUserMap) + public void handleAuthenticatorUserMapUpdate(String authenticatorPrefix, byte[] serializedUserMap) { } @@ -64,6 +67,18 @@ public Map getUserMap(String authenticatorPrefix "userA", new BasicAuthenticatorUser("userA", USER_A_CREDENTIALS) ); } + + @Override + public void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig) + { + + } + + @Override + public BasicAuthConfig getConfig(String authenticatorPrefix) + { + return null; + } } ); @@ -74,10 +89,22 @@ public Map getUserMap(String authenticatorPrefix new DefaultPasswordProvider("a"), new DefaultPasswordProvider("a"), false, + null, null, + false, + null, + null, + null, + null, + null, + null, + null, + null, + null, null, null ); + @Test public void testGoodPassword() throws IOException, ServletException { @@ -107,6 +134,68 @@ public void testGoodPassword() throws IOException, ServletException EasyMock.verify(req, resp, filterChain); } + @Test + public void testGoodPasswordWithValidator() throws IOException, ServletException + { + CredentialsValidator validator = EasyMock.createMock(CredentialsValidator.class); + BasicHTTPAuthenticator authenticatorWithValidator = new BasicHTTPAuthenticator( + CACHE_MANAGER_PROVIDER, + "basic", + "basic", + null, + null, + false, + null, null, + false, + validator, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + + String header = StringUtils.utf8Base64("userA:helloworld"); + header = StringUtils.format("Basic %s", header); + + EasyMock + .expect( + validator.validateCredentials(EasyMock.eq("basic"), EasyMock.eq("basic"), EasyMock.eq("userA"), EasyMock.aryEq("helloworld".toCharArray())) + ) + .andReturn( + new AuthenticationResult("userA", "basic", "basic", null) + ) + .times(1); + EasyMock.replay(validator); + + HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class); + EasyMock.expect(req.getHeader("Authorization")).andReturn(header); + req.setAttribute( + AuthConfig.DRUID_AUTHENTICATION_RESULT, + new AuthenticationResult("userA", "basic", "basic", null) + ); + EasyMock.expectLastCall().times(1); + EasyMock.replay(req); + + HttpServletResponse resp = EasyMock.createMock(HttpServletResponse.class); + EasyMock.replay(resp); + + FilterChain filterChain = EasyMock.createMock(FilterChain.class); + filterChain.doFilter(req, resp); + EasyMock.expectLastCall().times(1); + EasyMock.replay(filterChain); + + Filter authenticatorFilter = authenticatorWithValidator.getFilter(); + authenticatorFilter.doFilter(req, resp, filterChain); + + EasyMock.verify(req, resp, validator, filterChain); + } + @Test public void testBadPassword() throws IOException, ServletException { @@ -118,7 +207,7 @@ public void testBadPassword() throws IOException, ServletException EasyMock.replay(req); HttpServletResponse resp = EasyMock.createMock(HttpServletResponse.class); - resp.sendError(HttpServletResponse.SC_UNAUTHORIZED); + resp.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User DB authentication failed username[userA]."); EasyMock.expectLastCall().times(1); EasyMock.replay(resp); @@ -131,6 +220,63 @@ public void testBadPassword() throws IOException, ServletException EasyMock.verify(req, resp, filterChain); } + @Test + public void testBadPasswordWithSkipOnFailureValidator() throws IOException, ServletException + { + CredentialsValidator validator = EasyMock.createMock(CredentialsValidator.class); + BasicHTTPAuthenticator authenticatorWithValidator = new BasicHTTPAuthenticator( + CACHE_MANAGER_PROVIDER, + "basic", + "basic", + null, + null, + false, + null, null, + true, + validator, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + String header = StringUtils.utf8Base64("userA:badpassword"); + header = StringUtils.format("Basic %s", header); + + EasyMock + .expect( + validator.validateCredentials(EasyMock.eq("basic"), EasyMock.eq("basic"), EasyMock.eq("userA"), EasyMock.aryEq("badpassword".toCharArray())) + ) + .andThrow( + new BasicSecurityAuthenticationException("User authentication failed username[%s].", "userA") + ) + .times(1); + EasyMock.replay(validator); + + HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class); + EasyMock.expect(req.getHeader("Authorization")).andReturn(header); + EasyMock.replay(req); + + HttpServletResponse resp = EasyMock.createMock(HttpServletResponse.class); + resp.sendError(HttpServletResponse.SC_UNAUTHORIZED, "User authentication failed username[userA]."); + EasyMock.expectLastCall().times(1); + EasyMock.replay(resp); + + // Authentication filter should not move on to the next filter in the chain + FilterChain filterChain = EasyMock.createMock(FilterChain.class); + EasyMock.replay(filterChain); + + Filter authenticatorFilter = authenticatorWithValidator.getFilter(); + authenticatorFilter.doFilter(req, resp, filterChain); + + EasyMock.verify(req, resp, validator, filterChain); + } + @Test public void testUnknownUser() throws IOException, ServletException { @@ -155,6 +301,61 @@ public void testUnknownUser() throws IOException, ServletException EasyMock.verify(req, resp, filterChain); } + @Test + public void testUnknownUserWithSkipOnFailure() throws IOException, ServletException + { + CredentialsValidator validator = EasyMock.createMock(CredentialsValidator.class); + BasicHTTPAuthenticator authenticatorWithSkipOnFailure = new BasicHTTPAuthenticator( + CACHE_MANAGER_PROVIDER, + "basic", + "basic", + null, + null, + false, + null, null, + true, + validator, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + ); + String header = StringUtils.utf8Base64("userB:helloworld"); + header = StringUtils.format("Basic %s", header); + + HttpServletRequest req = EasyMock.createMock(HttpServletRequest.class); + EasyMock.expect(req.getHeader("Authorization")).andReturn(header); + EasyMock.replay(req); + + EasyMock + .expect( + validator.validateCredentials(EasyMock.eq("basic"), EasyMock.eq("basic"), EasyMock.eq("userB"), EasyMock.aryEq("helloworld".toCharArray())) + ) + .andReturn(null) + .times(1); + EasyMock.replay(validator); + + HttpServletResponse resp = EasyMock.createMock(HttpServletResponse.class); + EasyMock.replay(resp); + + // Authentication filter should move on to the next filter in the chain without sending a response + FilterChain filterChain = EasyMock.createMock(FilterChain.class); + filterChain.doFilter(req, resp); + EasyMock.expectLastCall().times(1); + EasyMock.replay(filterChain); + + Filter authenticatorFilter = authenticatorWithSkipOnFailure.getFilter(); + authenticatorFilter.doFilter(req, resp, filterChain); + + EasyMock.verify(req, resp, validator, filterChain); + } + @Test public void testRecognizedButMalformedBasicAuthHeader() throws IOException, ServletException { diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java index de566573a311..2360e72ea274 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java @@ -29,6 +29,7 @@ import org.apache.druid.security.basic.BasicSecurityDBResourceException; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.updater.CoordinatorBasicAuthenticatorMetadataStorageUpdater; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -45,6 +46,7 @@ public class CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest { private static final String AUTHENTICATOR_NAME = "test"; + @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -63,7 +65,6 @@ public void setUp() connector = derbyConnectorRule.getConnector(); tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); connector.createConfigTable(); - updater = new CoordinatorBasicAuthenticatorMetadataStorageUpdater( new AuthenticatorMapper( ImmutableMap.of( @@ -76,6 +77,18 @@ public void setUp() null, null, null, + null, + false, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, null ) ) @@ -100,16 +113,24 @@ public void tearDown() @Test public void createUser() { - updater.createUser(AUTHENTICATOR_NAME, "druid"); Map expectedUserMap = ImmutableMap.of( "druid", new BasicAuthenticatorUser("druid", null) ); + byte[] expectedSerializeUserMap = BasicAuthUtils.serializeAuthenticatorUserMap(objectMapper, expectedUserMap); + + updater.createUser(AUTHENTICATOR_NAME, "druid"); + Assert.assertArrayEquals(expectedSerializeUserMap, updater.getCurrentUserMapBytes(AUTHENTICATOR_NAME)); + Map actualUserMap = BasicAuthUtils.deserializeAuthenticatorUserMap( objectMapper, updater.getCurrentUserMapBytes(AUTHENTICATOR_NAME) ); Assert.assertEquals(expectedUserMap, actualUserMap); + // Validate cache user map methods + Assert.assertEquals(expectedUserMap, updater.getCachedUserMap(AUTHENTICATOR_NAME)); + Assert.assertArrayEquals(expectedSerializeUserMap, updater.getCachedSerializedUserMap(AUTHENTICATOR_NAME)); + // create duplicate should fail expectedException.expect(BasicSecurityDBResourceException.class); expectedException.expectMessage("User [druid] already exists."); @@ -119,15 +140,24 @@ public void createUser() @Test public void deleteUser() { + Map expectedUserMap = ImmutableMap.of(); + byte[] expectedSerializeUserMap = BasicAuthUtils.serializeAuthenticatorUserMap(objectMapper, expectedUserMap); + updater.createUser(AUTHENTICATOR_NAME, "druid"); updater.deleteUser(AUTHENTICATOR_NAME, "druid"); - Map expectedUserMap = ImmutableMap.of(); + + Assert.assertArrayEquals(expectedSerializeUserMap, updater.getCurrentUserMapBytes(AUTHENTICATOR_NAME)); + Map actualUserMap = BasicAuthUtils.deserializeAuthenticatorUserMap( objectMapper, updater.getCurrentUserMapBytes(AUTHENTICATOR_NAME) ); Assert.assertEquals(expectedUserMap, actualUserMap); + // Validate cache user map methods + Assert.assertEquals(expectedUserMap, updater.getCachedUserMap(AUTHENTICATOR_NAME)); + Assert.assertArrayEquals(expectedSerializeUserMap, updater.getCachedSerializedUserMap(AUTHENTICATOR_NAME)); + // delete non-existent user should fail expectedException.expect(BasicSecurityDBResourceException.class); expectedException.expectMessage("User [druid] does not exist."); @@ -153,6 +183,63 @@ public void setCredentials() ); Assert.assertArrayEquals(credentials.getHash(), recalculatedHash); + + // Validate cache user map methods + Map expectedUserMap = ImmutableMap.of( + "druid", new BasicAuthenticatorUser("druid", credentials) + ); + byte[] expectedSerializeUserMap = BasicAuthUtils.serializeAuthenticatorUserMap(objectMapper, expectedUserMap); + Assert.assertArrayEquals(expectedSerializeUserMap, updater.getCurrentUserMapBytes(AUTHENTICATOR_NAME)); + Assert.assertEquals(expectedUserMap, updater.getCachedUserMap(AUTHENTICATOR_NAME)); + Assert.assertArrayEquals(expectedSerializeUserMap, updater.getCachedSerializedUserMap(AUTHENTICATOR_NAME)); + } + + @Test + public void updateConfig() + { + BasicAuthConfig config = new BasicAuthConfig( + "https://testUrl", + "testUser", + "testPassword", + "testDn", + "testUserSearch", + "testUserAttribute", + new String[]{"testGroupFilter"} + ); + byte[] serializedConfig = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); + + updater.updateConfig(AUTHENTICATOR_NAME, config); + + Assert.assertArrayEquals(serializedConfig, updater.getCurrentConfigBytes(AUTHENTICATOR_NAME)); + + BasicAuthConfig actualConfig = BasicAuthUtils.deserializeAuthenticatorConfig( + objectMapper, + updater.getCurrentConfigBytes(AUTHENTICATOR_NAME) + ); + Assert.assertNotNull(actualConfig); + Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); + Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); + Assert.assertEquals(config.getBindPassword(), actualConfig.getBindPassword()); + Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); + Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); + Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); + Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); + + // Validate cache config methods + BasicAuthConfig cachedConfig = updater.getCachedConfig(AUTHENTICATOR_NAME); + Assert.assertNotNull(actualConfig); + Assert.assertEquals(config.getUrl(), cachedConfig.getUrl()); + Assert.assertEquals(config.getBindUser(), cachedConfig.getBindUser()); + Assert.assertEquals(config.getBindPassword(), cachedConfig.getBindPassword()); + Assert.assertEquals(config.getBaseDn(), cachedConfig.getBaseDn()); + Assert.assertEquals(config.getUserSearch(), cachedConfig.getUserSearch()); + Assert.assertEquals(config.getUserAttribute(), cachedConfig.getUserAttribute()); + Assert.assertArrayEquals(config.getGroupFilters(), cachedConfig.getGroupFilters()); + + Assert.assertArrayEquals(serializedConfig, updater.getCachedSerializedConfig(AUTHENTICATOR_NAME)); + + // update duplicate should not fail + updater.updateConfig(AUTHENTICATOR_NAME, config); } } diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java index dff71e376de0..54a99909f85f 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java @@ -32,6 +32,7 @@ import org.apache.druid.security.basic.authentication.db.updater.CoordinatorBasicAuthenticatorMetadataStorageUpdater; import org.apache.druid.security.basic.authentication.endpoint.BasicAuthenticatorResource; import org.apache.druid.security.basic.authentication.endpoint.CoordinatorBasicAuthenticatorResourceHandler; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -53,6 +54,7 @@ public class CoordinatorBasicAuthenticatorResourceTest { private static final String AUTHENTICATOR_NAME = "test"; private static final String AUTHENTICATOR_NAME2 = "test2"; + private static final String AUTHENTICATOR_NAME_LDAP = "testLdap"; @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -65,12 +67,14 @@ public class CoordinatorBasicAuthenticatorResourceTest private BasicAuthenticatorResource resource; private CoordinatorBasicAuthenticatorMetadataStorageUpdater storageUpdater; private HttpServletRequest req; + private ObjectMapper objectMapper; @Before public void setUp() { req = EasyMock.createStrictMock(HttpServletRequest.class); + objectMapper = new ObjectMapper(new SmileFactory()); connector = derbyConnectorRule.getConnector(); tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); connector.createConfigTable(); @@ -83,22 +87,53 @@ public void setUp() new BasicHTTPAuthenticator( null, AUTHENTICATOR_NAME, - "test", + null, new DefaultPasswordProvider("druid"), new DefaultPasswordProvider("druid"), null, null, + null, + false, + null, + null, null, null, null, null, null, null, null, null, null ), AUTHENTICATOR_NAME2, new BasicHTTPAuthenticator( null, AUTHENTICATOR_NAME2, - "test", + null, + new DefaultPasswordProvider("druid"), + new DefaultPasswordProvider("druid"), + null, + null, + null, + false, + null, + null, null, null, null, null, null, null, null, null, + null + ), + AUTHENTICATOR_NAME_LDAP, + new BasicHTTPAuthenticator( + null, + AUTHENTICATOR_NAME2, + null, new DefaultPasswordProvider("druid"), new DefaultPasswordProvider("druid"), null, null, + null, + false, + null, + "https://testUrl", + "testUser", + new DefaultPasswordProvider("testPassword"), + "testDn", + "testUserSearch", + "testUserAttribute", + new String[]{"testGroupFilter"}, + null, + null, null ) ) @@ -164,10 +199,26 @@ public void testGetAllUsers() response = resource.getAllUsers(req, AUTHENTICATOR_NAME); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(expectedUsers, response.getEntity()); + + // Verify cached user map is also getting updated + response = resource.getCachedSerializedUserMap(req, AUTHENTICATOR_NAME); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof byte[]); + Map cachedUserMap = BasicAuthUtils.deserializeAuthenticatorUserMap(objectMapper, (byte[]) response.getEntity()); + Assert.assertNotNull(cachedUserMap.get(BasicAuthUtils.ADMIN_NAME)); + Assert.assertEquals(cachedUserMap.get(BasicAuthUtils.ADMIN_NAME).getName(), BasicAuthUtils.ADMIN_NAME); + Assert.assertNotNull(cachedUserMap.get(BasicAuthUtils.INTERNAL_USER_NAME)); + Assert.assertEquals(cachedUserMap.get(BasicAuthUtils.ADMIN_NAME).getName(), BasicAuthUtils.ADMIN_NAME); + Assert.assertNotNull(cachedUserMap.get("druid")); + Assert.assertEquals(cachedUserMap.get("druid").getName(), "druid"); + Assert.assertNotNull(cachedUserMap.get("druid2")); + Assert.assertEquals(cachedUserMap.get("druid2").getName(), "druid2"); + Assert.assertNotNull(cachedUserMap.get("druid3")); + Assert.assertEquals(cachedUserMap.get("druid3").getName(), "druid3"); } @Test - public void testSeparateDatabaseTables() + public void testGetAllUsersSeparateDatabaseTables() { Response response = resource.getAllUsers(req, AUTHENTICATOR_NAME); Assert.assertEquals(200, response.getStatus()); @@ -201,9 +252,43 @@ public void testSeparateDatabaseTables() Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(expectedUsers, response.getEntity()); + // Verify cached user map for AUTHENTICATOR_NAME authenticator is also getting updated + response = resource.getCachedSerializedUserMap(req, AUTHENTICATOR_NAME); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof byte[]); + + Map cachedUserMap = BasicAuthUtils.deserializeAuthenticatorUserMap(objectMapper, (byte[]) response.getEntity()); + Assert.assertNotNull(cachedUserMap.get(BasicAuthUtils.ADMIN_NAME)); + Assert.assertEquals(cachedUserMap.get(BasicAuthUtils.ADMIN_NAME).getName(), BasicAuthUtils.ADMIN_NAME); + Assert.assertNotNull(cachedUserMap.get(BasicAuthUtils.INTERNAL_USER_NAME)); + Assert.assertEquals(cachedUserMap.get(BasicAuthUtils.ADMIN_NAME).getName(), BasicAuthUtils.ADMIN_NAME); + Assert.assertNotNull(cachedUserMap.get("druid")); + Assert.assertEquals(cachedUserMap.get("druid").getName(), "druid"); + Assert.assertNotNull(cachedUserMap.get("druid2")); + Assert.assertEquals(cachedUserMap.get("druid2").getName(), "druid2"); + Assert.assertNotNull(cachedUserMap.get("druid3")); + Assert.assertEquals(cachedUserMap.get("druid3").getName(), "druid3"); + response = resource.getAllUsers(req, AUTHENTICATOR_NAME2); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(expectedUsers2, response.getEntity()); + + // Verify cached user map for each AUTHENTICATOR_NAME2 is also getting updated + response = resource.getCachedSerializedUserMap(req, AUTHENTICATOR_NAME2); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof byte[]); + + cachedUserMap = BasicAuthUtils.deserializeAuthenticatorUserMap(objectMapper, (byte[]) response.getEntity()); + Assert.assertNotNull(cachedUserMap.get(BasicAuthUtils.ADMIN_NAME)); + Assert.assertEquals(cachedUserMap.get(BasicAuthUtils.ADMIN_NAME).getName(), BasicAuthUtils.ADMIN_NAME); + Assert.assertNotNull(cachedUserMap.get(BasicAuthUtils.INTERNAL_USER_NAME)); + Assert.assertEquals(cachedUserMap.get(BasicAuthUtils.ADMIN_NAME).getName(), BasicAuthUtils.ADMIN_NAME); + Assert.assertNotNull(cachedUserMap.get("druid4")); + Assert.assertEquals(cachedUserMap.get("druid4").getName(), "druid4"); + Assert.assertNotNull(cachedUserMap.get("druid5")); + Assert.assertEquals(cachedUserMap.get("druid5").getName(), "druid5"); + Assert.assertNotNull(cachedUserMap.get("druid6")); + Assert.assertEquals(cachedUserMap.get("druid6").getName(), "druid6"); } @Test @@ -220,6 +305,13 @@ public void testCreateDeleteUser() response = resource.deleteUser(req, AUTHENTICATOR_NAME, "druid"); Assert.assertEquals(200, response.getStatus()); + response = resource.getCachedSerializedUserMap(req, AUTHENTICATOR_NAME); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof byte[]); + Map cachedUserMap = BasicAuthUtils.deserializeAuthenticatorUserMap(objectMapper, (byte[]) response.getEntity()); + Assert.assertNotNull(cachedUserMap); + Assert.assertNull(cachedUserMap.get("druid")); + response = resource.deleteUser(req, AUTHENTICATOR_NAME, "druid"); Assert.assertEquals(400, response.getStatus()); Assert.assertEquals(errorMapWithMsg("User [druid] does not exist."), response.getEntity()); @@ -263,6 +355,29 @@ public void testUserCredentials() ); Assert.assertArrayEquals(recalculatedHash, hash); + response = resource.getCachedSerializedUserMap(req, AUTHENTICATOR_NAME); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof byte[]); + Map cachedUserMap = BasicAuthUtils.deserializeAuthenticatorUserMap(objectMapper, (byte[]) response.getEntity()); + Assert.assertNotNull(cachedUserMap); + Assert.assertNotNull(cachedUserMap.get("druid")); + Assert.assertEquals("druid", cachedUserMap.get("druid").getName()); + BasicAuthenticatorCredentials cachedUserCredentials = cachedUserMap.get("druid").getCredentials(); + + salt = cachedUserCredentials.getSalt(); + hash = cachedUserCredentials.getHash(); + iterations = cachedUserCredentials.getIterations(); + Assert.assertEquals(BasicAuthUtils.SALT_LENGTH, salt.length); + Assert.assertEquals(BasicAuthUtils.KEY_LENGTH / 8, hash.length); + Assert.assertEquals(BasicAuthUtils.DEFAULT_KEY_ITERATIONS, iterations); + + recalculatedHash = BasicAuthUtils.hashPassword( + "helloworld".toCharArray(), + salt, + iterations + ); + Assert.assertArrayEquals(recalculatedHash, hash); + response = resource.deleteUser(req, AUTHENTICATOR_NAME, "druid"); Assert.assertEquals(200, response.getStatus()); @@ -280,9 +395,131 @@ public void testUserCredentials() Assert.assertEquals(errorMapWithMsg("User [druid] does not exist."), response.getEntity()); } + @Test + public void testGetConfig() + { + BasicAuthConfig config = new BasicAuthConfig( + "https://testUrl", + "testUser", + "testPassword", + "testDn", + "testUserSearch", + "testUserAttribute", + new String[]{"testGroupFilter"} + ); + byte[] serializedConfig = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); + + Response response = resource.getConfig(req, AUTHENTICATOR_NAME_LDAP); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof BasicAuthConfig); + + BasicAuthConfig actualConfig = (BasicAuthConfig) response.getEntity(); + Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); + Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); + Assert.assertEquals("...", actualConfig.getBindPassword()); + Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); + Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); + Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); + Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); + + response = resource.getCachedSerializedConfig(req, AUTHENTICATOR_NAME_LDAP); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof byte[]); + Assert.assertArrayEquals(serializedConfig, (byte[]) response.getEntity()); + } + + @Test + public void testGetConfigSeparateDatabaseTables() + { + BasicAuthConfig config = new BasicAuthConfig( + null, + null, + null, + null, + null, + null, + null + ); + byte[] serializedConfig = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); + + Response response = resource.getConfig(req, AUTHENTICATOR_NAME2); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof BasicAuthConfig); + + BasicAuthConfig actualConfig = (BasicAuthConfig) response.getEntity(); + Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); + Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); + Assert.assertEquals(config.getBindPassword(), actualConfig.getBindPassword()); + Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); + Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); + Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); + Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); + + response = resource.getCachedSerializedConfig(req, AUTHENTICATOR_NAME2); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof byte[]); + Assert.assertArrayEquals(serializedConfig, (byte[]) response.getEntity()); + } + + @Test + public void testUpdateConfig() + { + BasicAuthConfig config = new BasicAuthConfig( + "https://testUrlUpdate", + "testUserUpdate", + "testPasswordUpdate", + "testDnUpdate", + "testUserSearchUpdate", + "testUserAttributeUpdate", + new String[]{"testGroupFilterUpdate"} + ); + byte[] serializedConfig = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); + + Response response = resource.updateConfig(req, AUTHENTICATOR_NAME_LDAP, config); + Assert.assertEquals(200, response.getStatus()); + + response = resource.getConfig(req, AUTHENTICATOR_NAME_LDAP); + Assert.assertTrue(response.getEntity() instanceof BasicAuthConfig); + + BasicAuthConfig actualConfig = (BasicAuthConfig) response.getEntity(); + Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); + Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); + Assert.assertEquals("...", actualConfig.getBindPassword()); + Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); + Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); + Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); + Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); + + response = resource.getCachedSerializedConfig(req, AUTHENTICATOR_NAME_LDAP); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof byte[]); + Assert.assertArrayEquals(serializedConfig, (byte[]) response.getEntity()); + + // Verify other authenticator config is not getting updated + config = new BasicAuthConfig( + null, + null, + null, + null, + null, + null, + null + ); + response = resource.getConfig(req, AUTHENTICATOR_NAME2); + Assert.assertEquals(200, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof BasicAuthConfig); + actualConfig = (BasicAuthConfig) response.getEntity(); + Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); + Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); + Assert.assertEquals(config.getBindPassword(), actualConfig.getBindPassword()); + Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); + Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); + Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); + Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); + } + private static Map errorMapWithMsg(String errorMsg) { return ImmutableMap.of("error", errorMsg); } - } diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java index 7c03264bf4c0..c7a1f265b31b 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java @@ -24,7 +24,12 @@ public class NoopBasicAuthenticatorCacheNotifier implements BasicAuthenticatorCacheNotifier { @Override - public void addUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserMap) + public void addUserUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserMap) + { + } + + @Override + public void addConfigUpdate(String updatedAuthenticatorPrefix, byte[] updatedConfig) { } diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java new file mode 100644 index 000000000000..7a207504acf3 --- /dev/null +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java @@ -0,0 +1,165 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.security.authentication.validator; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Provider; +import com.google.inject.util.Providers; +import org.apache.druid.java.util.common.IAE; +import org.apache.druid.security.basic.BasicSecurityAuthenticationException; +import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; +import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; +import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; +import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; +import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; +import org.apache.druid.security.basic.authentication.validator.DBCredentialsValidator; +import org.apache.druid.server.security.AuthenticationResult; +import org.easymock.EasyMock; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +public class DBCredentialsValidatorTest +{ + + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + public static BasicAuthenticatorCredentials USER_A_CREDENTIALS = new BasicAuthenticatorCredentials( + new BasicAuthenticatorCredentialUpdate("helloworld", 20) + ); + + public static Provider CACHE_MANAGER_PROVIDER = Providers.of( + new BasicAuthenticatorCacheManager() + { + @Override + public void handleAuthenticatorUserMapUpdate(String authenticatorPrefix, byte[] serializedUserMap) + { + + } + + @Override + public Map getUserMap(String authenticatorPrefix) + { + return ImmutableMap.of( + "userA", new BasicAuthenticatorUser("userA", USER_A_CREDENTIALS), + "userB", new BasicAuthenticatorUser("userB", null) + ); + } + + @Override + public void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig) + { + + } + + @Override + public BasicAuthConfig getConfig(String authenticatorPrefix) + { + return null; + } + } + ); + + private static DBCredentialsValidator validator = new DBCredentialsValidator(CACHE_MANAGER_PROVIDER); + + + + @Test + public void validateBadAuthenticator() + { + String authenticatorName = "notbasic"; + String authorizerName = "basic"; + String username = "userA"; + String password = "helloworld"; + + BasicAuthenticatorCacheManager cacheManager = EasyMock.createMock(BasicAuthenticatorCacheManager.class); + EasyMock.expect(cacheManager.getUserMap(authenticatorName)).andReturn(null).times(1); + EasyMock.replay(cacheManager); + + DBCredentialsValidator validator = new DBCredentialsValidator(Providers.of(cacheManager)); + + expectedException.expect(IAE.class); + expectedException.expectMessage("No authenticator found with prefix: [notbasic]"); + validator.validateCredentials(authenticatorName, authorizerName, username, password.toCharArray()); + + EasyMock.verify(cacheManager); + } + + @Test + public void validateMissingCredentials() + { + String authenticatorName = "basic"; + String authorizerName = "basic"; + String username = "userB"; + String password = "helloworld"; + + AuthenticationResult result = validator.validateCredentials(authenticatorName, authorizerName, username, password.toCharArray()); + assertNull(result); + } + + @Test + public void validateMissingUser() + { + String authenticatorName = "basic"; + String authorizerName = "basic"; + String username = "userC"; + String password = "helloworld"; + + AuthenticationResult result = validator.validateCredentials(authenticatorName, authorizerName, username, password.toCharArray()); + assertNull(result); + } + + @Test + public void validateGoodCredentials() + { + String authenticatorName = "basic"; + String authorizerName = "basic"; + String username = "userA"; + String password = "helloworld"; + + AuthenticationResult result = validator.validateCredentials(authenticatorName, authorizerName, username, password.toCharArray()); + + assertNotNull(result); + assertEquals(username, result.getIdentity()); + assertEquals(authenticatorName, result.getAuthenticatedBy()); + assertEquals(authorizerName, result.getAuthorizerName()); + assertNull(result.getContext()); + } + + @Test + public void validateBadCredentials() + { + String authenticatorName = "basic"; + String authorizerName = "basic"; + String username = "userA"; + String password = "badpassword"; + + expectedException.expect(BasicSecurityAuthenticationException.class); + expectedException.expectMessage("User DB authentication failed username[userA]."); + validator.validateCredentials(authenticatorName, authorizerName, username, password.toCharArray()); + } +} diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/BasicRoleBasedAuthorizerTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/BasicRoleBasedAuthorizerTest.java index 53ef10bf74d2..abea81f264d8 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/BasicRoleBasedAuthorizerTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/BasicRoleBasedAuthorizerTest.java @@ -25,9 +25,11 @@ import org.apache.druid.metadata.MetadataStorageTablesConfig; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.security.basic.BasicAuthCommonCacheConfig; +import org.apache.druid.security.basic.BasicAuthUtils; import org.apache.druid.security.basic.authorization.BasicRoleBasedAuthorizer; import org.apache.druid.security.basic.authorization.db.cache.MetadataStoragePollingBasicAuthorizerCacheManager; import org.apache.druid.security.basic.authorization.db.updater.CoordinatorBasicAuthorizerMetadataStorageUpdater; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthenticationResult; @@ -41,8 +43,14 @@ import org.junit.Rule; import org.junit.Test; +import javax.naming.InvalidNameException; +import javax.naming.ldap.LdapName; +import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; public class BasicRoleBasedAuthorizerTest { @@ -71,6 +79,8 @@ public void setUp() null, AUTHORIZER_NAME, null, + null, + null, null, null ) ) @@ -91,6 +101,8 @@ public void setUp() ), AUTHORIZER_NAME, null, + null, + null, null, null ); } @@ -105,7 +117,7 @@ public void testAuth() { updater.createUser(AUTHORIZER_NAME, "druid"); updater.createRole(AUTHORIZER_NAME, "druidRole"); - updater.assignRole(AUTHORIZER_NAME, "druid", "druidRole"); + updater.assignUserRole(AUTHORIZER_NAME, "druid", "druidRole"); List permissions = Collections.singletonList( new ResourceAction(new Resource("testResource", ResourceType.DATASOURCE), Action.WRITE) @@ -129,4 +141,215 @@ public void testAuth() ); Assert.assertFalse(access.isAllowed()); } + + @Test + public void testAuthGroupMapping() throws InvalidNameException + { + BasicAuthorizerGroupMapping groupMapping = new BasicAuthorizerGroupMapping("druidGroupMapping", "CN=test", null); + updater.createGroupMapping(AUTHORIZER_NAME, groupMapping); + updater.createRole(AUTHORIZER_NAME, "druidRole"); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "druidGroupMapping", "druidRole"); + + List permissions = Collections.singletonList( + new ResourceAction(new Resource("testResource", ResourceType.DATASOURCE), Action.WRITE) + ); + + updater.setPermissions(AUTHORIZER_NAME, "druidRole", permissions); + + Map contexMap = new HashMap<>(); + contexMap.put(BasicAuthUtils.GROUPS_CONTEXT_KEY, Collections.singleton(new LdapName("CN=test"))); + + AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid", null, contexMap); + + Access access = authorizer.authorize( + authenticationResult, + new Resource("testResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertTrue(access.isAllowed()); + + access = authorizer.authorize( + authenticationResult, + new Resource("wrongResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + } + + @Test + public void testAuthGroupMappingPatternRightMask() throws InvalidNameException + { + BasicAuthorizerGroupMapping groupMapping = new BasicAuthorizerGroupMapping("druidGroupMapping", "CN=test,*", null); + updater.createGroupMapping(AUTHORIZER_NAME, groupMapping); + updater.createRole(AUTHORIZER_NAME, "druidRole"); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "druidGroupMapping", "druidRole"); + + List permissions = Collections.singletonList( + new ResourceAction(new Resource("testResource", ResourceType.DATASOURCE), Action.WRITE) + ); + + updater.setPermissions(AUTHORIZER_NAME, "druidRole", permissions); + + Map contexMap = new HashMap<>(); + + contexMap.put(BasicAuthUtils.GROUPS_CONTEXT_KEY, Collections.singleton(new LdapName("CN=test,ou=groupings,dc=corp"))); + AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid", null, contexMap); + + Access access = authorizer.authorize( + authenticationResult, + new Resource("testResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertTrue(access.isAllowed()); + + access = authorizer.authorize( + authenticationResult, + new Resource("wrongResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + + contexMap = new HashMap<>(); + contexMap.put(BasicAuthUtils.GROUPS_CONTEXT_KEY, Collections.singleton(new LdapName("CN=test"))); + authenticationResult = new AuthenticationResult("druid", "druid", null, contexMap); + + access = authorizer.authorize( + authenticationResult, + new Resource("testResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertTrue(access.isAllowed()); + + access = authorizer.authorize( + authenticationResult, + new Resource("wrongResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + + contexMap = new HashMap<>(); + contexMap.put(BasicAuthUtils.GROUPS_CONTEXT_KEY, Collections.singleton(new LdapName("CN=druid,CN=test"))); + authenticationResult = new AuthenticationResult("druid", "druid", null, contexMap); + + access = authorizer.authorize( + authenticationResult, + new Resource("testResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + + access = authorizer.authorize( + authenticationResult, + new Resource("wrongResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + } + + @Test + public void testAuthGroupMappingPatternLeftMask() throws InvalidNameException + { + BasicAuthorizerGroupMapping groupMapping = new BasicAuthorizerGroupMapping("druidGroupMapping", "*,CN=test", null); + updater.createGroupMapping(AUTHORIZER_NAME, groupMapping); + updater.createRole(AUTHORIZER_NAME, "druidRole"); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "druidGroupMapping", "druidRole"); + + List permissions = Collections.singletonList( + new ResourceAction(new Resource("testResource", ResourceType.DATASOURCE), Action.WRITE) + ); + + updater.setPermissions(AUTHORIZER_NAME, "druidRole", permissions); + + Map contexMap = new HashMap<>(); + + contexMap.put(BasicAuthUtils.GROUPS_CONTEXT_KEY, Collections.singleton(new LdapName("CN=druid,CN=test"))); + AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid", null, contexMap); + + Access access = authorizer.authorize( + authenticationResult, + new Resource("testResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertTrue(access.isAllowed()); + + access = authorizer.authorize( + authenticationResult, + new Resource("wrongResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + + contexMap = new HashMap<>(); + contexMap.put(BasicAuthUtils.GROUPS_CONTEXT_KEY, Collections.singleton(new LdapName("CN=test"))); + authenticationResult = new AuthenticationResult("druid", "druid", null, contexMap); + + access = authorizer.authorize( + authenticationResult, + new Resource("testResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertTrue(access.isAllowed()); + + access = authorizer.authorize( + authenticationResult, + new Resource("wrongResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + + contexMap = new HashMap<>(); + contexMap.put(BasicAuthUtils.GROUPS_CONTEXT_KEY, Collections.singleton(new LdapName("CN=test,CN=druid"))); + authenticationResult = new AuthenticationResult("druid", "druid", null, contexMap); + + access = authorizer.authorize( + authenticationResult, + new Resource("testResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + + access = authorizer.authorize( + authenticationResult, + new Resource("wrongResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + } + + @Test + public void testAuthMissingGroupMapping() throws InvalidNameException + { + BasicAuthorizerGroupMapping groupMapping = new BasicAuthorizerGroupMapping("druidGroupMapping", "CN=test", null); + updater.createGroupMapping(AUTHORIZER_NAME, groupMapping); + updater.createRole(AUTHORIZER_NAME, "druidRole"); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "druidGroupMapping", "druidRole"); + + List permissions = Collections.singletonList( + new ResourceAction(new Resource("testResource", ResourceType.DATASOURCE), Action.WRITE) + ); + + updater.setPermissions(AUTHORIZER_NAME, "druidRole", permissions); + + Map contexMap = new HashMap<>(); + contexMap.put( + BasicAuthUtils.GROUPS_CONTEXT_KEY, + new HashSet<>(Arrays.asList(new LdapName("CN=unknown"), new LdapName("CN=test,ou=groupings,dc=corp"))) + ); + + AuthenticationResult authenticationResult = new AuthenticationResult("druid", "druid", null, contexMap); + + Access access = authorizer.authorize( + authenticationResult, + new Resource("testResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + + access = authorizer.authorize( + authenticationResult, + new Resource("wrongResource", ResourceType.DATASOURCE), + Action.WRITE + ); + Assert.assertFalse(access.isAllowed()); + } } diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java index 5b565e177a9b..31748006d6c4 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java @@ -31,6 +31,7 @@ import org.apache.druid.security.basic.BasicSecurityDBResourceException; import org.apache.druid.security.basic.authorization.BasicRoleBasedAuthorizer; import org.apache.druid.security.basic.authorization.db.updater.CoordinatorBasicAuthorizerMetadataStorageUpdater; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerPermission; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; @@ -101,6 +102,8 @@ public void setUp() null, AUTHORIZER_NAME, null, + null, + null, null, null ) ) @@ -138,6 +141,27 @@ public void testCreateDeleteUser() Assert.assertEquals(expectedUserMap, actualUserMap); } + @Test + public void testCreateDeleteGroupMapping() + { + updater.createGroupMapping(AUTHORIZER_NAME, new BasicAuthorizerGroupMapping("druid", "CN=test", null)); + Map expectedGroupMappingMap = new HashMap<>(); + expectedGroupMappingMap.put("druid", new BasicAuthorizerGroupMapping("druid", "CN=test", null)); + Map actualGroupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + updater.getCurrentGroupMappingMapBytes(AUTHORIZER_NAME) + ); + Assert.assertEquals(expectedGroupMappingMap, actualGroupMappingMap); + + updater.deleteGroupMapping(AUTHORIZER_NAME, "druid"); + expectedGroupMappingMap.remove("druid"); + actualGroupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + updater.getCurrentGroupMappingMapBytes(AUTHORIZER_NAME) + ); + Assert.assertEquals(expectedGroupMappingMap, actualGroupMappingMap); + } + @Test public void testDeleteNonExistentUser() { @@ -146,6 +170,15 @@ public void testDeleteNonExistentUser() updater.deleteUser(AUTHORIZER_NAME, "druid"); } + @Test + public void testDeleteNonExistentGroupMapping() + { + expectedException.expect(BasicSecurityDBResourceException.class); + expectedException.expectMessage("Group mapping [druid] does not exist."); + updater.deleteGroupMapping(AUTHORIZER_NAME, "druid"); + } + + @Test public void testCreateDuplicateUser() { @@ -155,6 +188,14 @@ public void testCreateDuplicateUser() updater.createUser(AUTHORIZER_NAME, "druid"); } + @Test + public void testCreateDuplicateGroupMapping() + { + expectedException.expect(BasicSecurityDBResourceException.class); + expectedException.expectMessage("Group mapping [druid] already exists."); + updater.createGroupMapping(AUTHORIZER_NAME, new BasicAuthorizerGroupMapping("druid", "CN=test", null)); + updater.createGroupMapping(AUTHORIZER_NAME, new BasicAuthorizerGroupMapping("druid", "CN=test", null)); + } // role tests @Test public void testCreateDeleteRole() @@ -194,13 +235,13 @@ public void testCreateDuplicateRole() updater.createRole(AUTHORIZER_NAME, "druid"); } - // role and user tests + // role, user, and group mapping tests @Test - public void testAddAndRemoveRole() + public void testAddAndRemoveRoleToUser() { updater.createUser(AUTHORIZER_NAME, "druid"); updater.createRole(AUTHORIZER_NAME, "druidRole"); - updater.assignRole(AUTHORIZER_NAME, "druid", "druidRole"); + updater.assignUserRole(AUTHORIZER_NAME, "druid", "druidRole"); Map expectedUserMap = new HashMap<>(BASE_USER_MAP); expectedUserMap.put("druid", new BasicAuthorizerUser("druid", ImmutableSet.of("druidRole"))); @@ -221,7 +262,7 @@ public void testAddAndRemoveRole() Assert.assertEquals(expectedUserMap, actualUserMap); Assert.assertEquals(expectedRoleMap, actualRoleMap); - updater.unassignRole(AUTHORIZER_NAME, "druid", "druidRole"); + updater.unassignUserRole(AUTHORIZER_NAME, "druid", "druidRole"); expectedUserMap.put("druid", new BasicAuthorizerUser("druid", ImmutableSet.of())); actualUserMap = BasicAuthUtils.deserializeAuthorizerUserMap( objectMapper, @@ -232,13 +273,60 @@ public void testAddAndRemoveRole() Assert.assertEquals(expectedRoleMap, actualRoleMap); } + // role, user, and group mapping tests + @Test + public void testAddAndRemoveRoleToGroupMapping() + { + updater.createGroupMapping(AUTHORIZER_NAME, new BasicAuthorizerGroupMapping("druid", "CN=test", null)); + updater.createRole(AUTHORIZER_NAME, "druidRole"); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "druid", "druidRole"); + + Map expectedGroupMappingMap = new HashMap<>(); + expectedGroupMappingMap.put("druid", new BasicAuthorizerGroupMapping("druid", "CN=test", ImmutableSet.of("druidRole"))); + + Map expectedRoleMap = new HashMap<>(BASE_ROLE_MAP); + expectedRoleMap.put("druidRole", new BasicAuthorizerRole("druidRole", ImmutableList.of())); + + Map actualGroupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + updater.getCurrentGroupMappingMapBytes(AUTHORIZER_NAME) + ); + + Map actualRoleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( + objectMapper, + updater.getCurrentRoleMapBytes(AUTHORIZER_NAME) + ); + + Assert.assertEquals(expectedGroupMappingMap, actualGroupMappingMap); + Assert.assertEquals(expectedRoleMap, actualRoleMap); + + updater.unassignGroupMappingRole(AUTHORIZER_NAME, "druid", "druidRole"); + expectedGroupMappingMap.put("druid", new BasicAuthorizerGroupMapping("druid", "CN=test", ImmutableSet.of())); + actualGroupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + updater.getCurrentGroupMappingMapBytes(AUTHORIZER_NAME) + ); + + Assert.assertEquals(expectedGroupMappingMap, actualGroupMappingMap); + Assert.assertEquals(expectedRoleMap, actualRoleMap); + } + @Test public void testAddRoleToNonExistentUser() { expectedException.expect(BasicSecurityDBResourceException.class); expectedException.expectMessage("User [nonUser] does not exist."); updater.createRole(AUTHORIZER_NAME, "druid"); - updater.assignRole(AUTHORIZER_NAME, "nonUser", "druid"); + updater.assignUserRole(AUTHORIZER_NAME, "nonUser", "druid"); + } + + @Test + public void testAddRoleToNonExistentGroupMapping() + { + expectedException.expect(BasicSecurityDBResourceException.class); + expectedException.expectMessage("Group mapping [nonUser] does not exist."); + updater.createRole(AUTHORIZER_NAME, "druid"); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "nonUser", "druid"); } @Test @@ -247,7 +335,16 @@ public void testAddNonexistentRoleToUser() expectedException.expect(BasicSecurityDBResourceException.class); expectedException.expectMessage("Role [nonRole] does not exist."); updater.createUser(AUTHORIZER_NAME, "druid"); - updater.assignRole(AUTHORIZER_NAME, "druid", "nonRole"); + updater.assignUserRole(AUTHORIZER_NAME, "druid", "nonRole"); + } + + @Test + public void testAddNonexistentRoleToGroupMapping() + { + expectedException.expect(BasicSecurityDBResourceException.class); + expectedException.expectMessage("Role [nonRole] does not exist."); + updater.createGroupMapping(AUTHORIZER_NAME, new BasicAuthorizerGroupMapping("druid", "CN=test", null)); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "druid", "nonRole"); } @Test @@ -257,12 +354,33 @@ public void testAddExistingRoleToUserFails() expectedException.expectMessage("User [druid] already has role [druidRole]."); updater.createUser(AUTHORIZER_NAME, "druid"); updater.createRole(AUTHORIZER_NAME, "druidRole"); - updater.assignRole(AUTHORIZER_NAME, "druid", "druidRole"); - updater.assignRole(AUTHORIZER_NAME, "druid", "druidRole"); + updater.assignUserRole(AUTHORIZER_NAME, "druid", "druidRole"); + updater.assignUserRole(AUTHORIZER_NAME, "druid", "druidRole"); } @Test - public void testUnassignInvalidRoleAssignmentFails() + public void testAddExistingRoleToGroupMappingFails() + { + expectedException.expect(BasicSecurityDBResourceException.class); + expectedException.expectMessage("Group mapping [druid] already has role [druidRole]."); + updater.createGroupMapping(AUTHORIZER_NAME, new BasicAuthorizerGroupMapping("druid", "CN=test", null)); + updater.createRole(AUTHORIZER_NAME, "druidRole"); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "druid", "druidRole"); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "druid", "druidRole"); + } + + @Test + public void testAddExistingRoleToGroupMappingWithRoleFails() + { + expectedException.expect(BasicSecurityDBResourceException.class); + expectedException.expectMessage("Group mapping [druid] already has role [druidRole]."); + updater.createGroupMapping(AUTHORIZER_NAME, new BasicAuthorizerGroupMapping("druid", "CN=test", ImmutableSet.of("druidRole"))); + updater.createRole(AUTHORIZER_NAME, "druidRole"); + updater.assignGroupMappingRole(AUTHORIZER_NAME, "druid", "druidRole"); + } + + @Test + public void testUnassignInvalidRoleAssignmentToUserFails() { expectedException.expect(BasicSecurityDBResourceException.class); expectedException.expectMessage("User [druid] does not have role [druidRole]."); @@ -289,16 +407,49 @@ public void testUnassignInvalidRoleAssignmentFails() Assert.assertEquals(expectedUserMap, actualUserMap); Assert.assertEquals(expectedRoleMap, actualRoleMap); - updater.unassignRole(AUTHORIZER_NAME, "druid", "druidRole"); + updater.unassignUserRole(AUTHORIZER_NAME, "druid", "druidRole"); + } + + @Test + public void testUnassignInvalidRoleAssignmentToGroupMappingFails() + { + expectedException.expect(BasicSecurityDBResourceException.class); + expectedException.expectMessage("Group mapping [druid] does not have role [druidRole]."); + + + updater.createGroupMapping(AUTHORIZER_NAME, new BasicAuthorizerGroupMapping("druid", "CN=test", null)); + updater.createRole(AUTHORIZER_NAME, "druidRole"); + + Map expectedGroupMappingMap = new HashMap<>(); + expectedGroupMappingMap.put("druid", new BasicAuthorizerGroupMapping("druid", "CN=test", null)); + + Map expectedRoleMap = new HashMap<>(BASE_ROLE_MAP); + expectedRoleMap.put("druidRole", new BasicAuthorizerRole("druidRole", ImmutableList.of())); + + Map actualGroupMappingMap = BasicAuthUtils.deserializeAuthorizerGroupMappingMap( + objectMapper, + updater.getCurrentGroupMappingMapBytes(AUTHORIZER_NAME) + ); + + Map actualRoleMap = BasicAuthUtils.deserializeAuthorizerRoleMap( + objectMapper, + updater.getCurrentRoleMapBytes(AUTHORIZER_NAME) + ); + + Assert.assertEquals(expectedGroupMappingMap, actualGroupMappingMap); + Assert.assertEquals(expectedRoleMap, actualRoleMap); + + updater.unassignGroupMappingRole(AUTHORIZER_NAME, "druid", "druidRole"); } + // role and permission tests @Test public void testSetRolePermissions() { updater.createUser(AUTHORIZER_NAME, "druid"); updater.createRole(AUTHORIZER_NAME, "druidRole"); - updater.assignRole(AUTHORIZER_NAME, "druid", "druidRole"); + updater.assignUserRole(AUTHORIZER_NAME, "druid", "druidRole"); List permsToAdd = ImmutableList.of( new ResourceAction( diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java index ef10dcc30dee..70a4a70ec933 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java @@ -33,6 +33,8 @@ import org.apache.druid.security.basic.authorization.db.updater.CoordinatorBasicAuthorizerMetadataStorageUpdater; import org.apache.druid.security.basic.authorization.endpoint.BasicAuthorizerResource; import org.apache.druid.security.basic.authorization.endpoint.CoordinatorBasicAuthorizerResourceHandler; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; +import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMappingFull; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerPermission; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRoleFull; @@ -53,6 +55,7 @@ import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -61,6 +64,7 @@ public class CoordinatorBasicAuthorizerResourceTest { private static final String AUTHORIZER_NAME = "test"; private static final String AUTHORIZER_NAME2 = "test2"; + private static final String AUTHORIZER_NAME3 = "test3"; @Rule public ExpectedException expectedException = ExpectedException.none(); @@ -90,6 +94,9 @@ public void setUp() null, AUTHORIZER_NAME, null, + null, + null, + null, null ), AUTHORIZER_NAME2, @@ -97,6 +104,19 @@ public void setUp() null, AUTHORIZER_NAME2, null, + null, + null, + null, + null + ), + AUTHORIZER_NAME3, + new BasicRoleBasedAuthorizer( + null, + AUTHORIZER_NAME3, + null, + null, + "adminGroupMapping", + null, null ) ) @@ -139,13 +159,47 @@ public void testSeparateDatabaseTables() response.getEntity() ); + response = resource.getAllUsers(req, AUTHORIZER_NAME2); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableSet.of(BasicAuthUtils.ADMIN_NAME, BasicAuthUtils.INTERNAL_USER_NAME), + response.getEntity() + ); + + response = resource.getAllGroupMappings(req, AUTHORIZER_NAME); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableSet.of(), + response.getEntity() + ); + + response = resource.getAllGroupMappings(req, AUTHORIZER_NAME2); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableSet.of(), + response.getEntity() + ); + + response = resource.getAllGroupMappings(req, AUTHORIZER_NAME3); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableSet.of("adminGroupMapping"), + response.getEntity() + ); + resource.createUser(req, AUTHORIZER_NAME, "druid"); resource.createUser(req, AUTHORIZER_NAME, "druid2"); resource.createUser(req, AUTHORIZER_NAME, "druid3"); + resource.createGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", new BasicAuthorizerGroupMapping("druidGroupMapping", "", new HashSet<>())); + resource.createGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", new BasicAuthorizerGroupMapping("druid2GroupMapping", "", new HashSet<>())); + resource.createGroupMapping(req, AUTHORIZER_NAME, "druid3GroupMapping", new BasicAuthorizerGroupMapping("druid3GroupMapping", "", new HashSet<>())); resource.createUser(req, AUTHORIZER_NAME2, "druid4"); resource.createUser(req, AUTHORIZER_NAME2, "druid5"); resource.createUser(req, AUTHORIZER_NAME2, "druid6"); + resource.createGroupMapping(req, AUTHORIZER_NAME2, "druid4GroupMapping", new BasicAuthorizerGroupMapping("druid4GroupMapping", "", new HashSet<>())); + resource.createGroupMapping(req, AUTHORIZER_NAME2, "druid5GroupMapping", new BasicAuthorizerGroupMapping("druid5GroupMapping", "", new HashSet<>())); + resource.createGroupMapping(req, AUTHORIZER_NAME2, "druid6GroupMapping", new BasicAuthorizerGroupMapping("druid6GroupMapping", "", new HashSet<>())); Set expectedUsers = ImmutableSet.of( BasicAuthUtils.ADMIN_NAME, @@ -170,6 +224,26 @@ public void testSeparateDatabaseTables() response = resource.getAllUsers(req, AUTHORIZER_NAME2); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(expectedUsers2, response.getEntity()); + + Set expectedGroupMappings = ImmutableSet.of( + "druidGroupMapping", + "druid2GroupMapping", + "druid3GroupMapping" + ); + + Set expectedGroupMappings2 = ImmutableSet.of( + "druid4GroupMapping", + "druid5GroupMapping", + "druid6GroupMapping" + ); + + response = resource.getAllGroupMappings(req, AUTHORIZER_NAME); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(expectedGroupMappings, response.getEntity()); + + response = resource.getAllGroupMappings(req, AUTHORIZER_NAME2); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(expectedGroupMappings2, response.getEntity()); } @Test @@ -210,6 +284,31 @@ public void testGetAllUsers() Assert.assertEquals(expectedUsers, response.getEntity()); } + @Test + public void testGetAllGroupMappings() + { + Response response = resource.getAllGroupMappings(req, AUTHORIZER_NAME); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals( + ImmutableSet.of(), + response.getEntity() + ); + + resource.createGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", new BasicAuthorizerGroupMapping("druidGroupMapping", "", new HashSet<>())); + resource.createGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", new BasicAuthorizerGroupMapping("druid2GroupMapping", "", new HashSet<>())); + resource.createGroupMapping(req, AUTHORIZER_NAME, "druid3GroupMapping", new BasicAuthorizerGroupMapping("druid3GroupMapping", "", new HashSet<>())); + + Set expectedGroupMappings = ImmutableSet.of( + "druidGroupMapping", + "druid2GroupMapping", + "druid3GroupMapping" + ); + + response = resource.getAllGroupMappings(req, AUTHORIZER_NAME); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(expectedGroupMappings, response.getEntity()); + } + @Test public void testGetAllRoles() { @@ -264,6 +363,33 @@ public void testCreateDeleteUser() Assert.assertEquals(errorMapWithMsg("User [druid] does not exist."), response.getEntity()); } + @Test + public void testCreateDeleteGroupMapping() + { + Response response = resource.createGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", new BasicAuthorizerGroupMapping("druidGroupMapping", "", new HashSet<>())); + Assert.assertEquals(200, response.getStatus()); + + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", null); + Assert.assertEquals(200, response.getStatus()); + + BasicAuthorizerGroupMapping expectedGroupMapping = new BasicAuthorizerGroupMapping( + "druidGroupMapping", + "", ImmutableSet.of() + ); + Assert.assertEquals(expectedGroupMapping, response.getEntity()); + + response = resource.deleteGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping"); + Assert.assertEquals(200, response.getStatus()); + + response = resource.deleteGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping"); + Assert.assertEquals(400, response.getStatus()); + Assert.assertEquals(errorMapWithMsg("Group mapping [druidGroupMapping] does not exist."), response.getEntity()); + + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", null); + Assert.assertEquals(400, response.getStatus()); + Assert.assertEquals(errorMapWithMsg("Group mapping [druidGroupMapping] does not exist."), response.getEntity()); + } + @Test public void testCreateDeleteRole() { @@ -289,7 +415,7 @@ public void testCreateDeleteRole() } @Test - public void testRoleAssignment() + public void testUserRoleAssignment() { Response response = resource.createRole(req, AUTHORIZER_NAME, "druidRole"); Assert.assertEquals(200, response.getStatus()); @@ -330,6 +456,48 @@ public void testRoleAssignment() Assert.assertEquals(expectedRole, response.getEntity()); } + @Test + public void testGroupMappingRoleAssignment() + { + Response response = resource.createRole(req, AUTHORIZER_NAME, "druidRole"); + Assert.assertEquals(200, response.getStatus()); + + response = resource.createGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", new BasicAuthorizerGroupMapping("druidGroupMapping", "", new HashSet<>())); + Assert.assertEquals(200, response.getStatus()); + + response = resource.assignRoleToGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", "druidRole"); + Assert.assertEquals(200, response.getStatus()); + + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", null); + Assert.assertEquals(200, response.getStatus()); + + BasicAuthorizerGroupMapping expectedGroupMapping = new BasicAuthorizerGroupMapping( + "druidGroupMapping", + "", ImmutableSet.of("druidRole") + ); + Assert.assertEquals(expectedGroupMapping, response.getEntity()); + + response = resource.getRole(req, AUTHORIZER_NAME, "druidRole", null); + Assert.assertEquals(200, response.getStatus()); + BasicAuthorizerRole expectedRole = new BasicAuthorizerRole("druidRole", ImmutableList.of()); + Assert.assertEquals(expectedRole, response.getEntity()); + + response = resource.unassignRoleFromGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", "druidRole"); + Assert.assertEquals(200, response.getStatus()); + + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", null); + Assert.assertEquals(200, response.getStatus()); + expectedGroupMapping = new BasicAuthorizerGroupMapping( + "druidGroupMapping", + "", ImmutableSet.of() + ); + Assert.assertEquals(expectedGroupMapping, response.getEntity()); + + response = resource.getRole(req, AUTHORIZER_NAME, "druidRole", null); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(expectedRole, response.getEntity()); + } + @Test public void testDeleteAssignedRole() { @@ -348,6 +516,18 @@ public void testDeleteAssignedRole() response = resource.assignRoleToUser(req, AUTHORIZER_NAME, "druid2", "druidRole"); Assert.assertEquals(200, response.getStatus()); + response = resource.createGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", new BasicAuthorizerGroupMapping("druidGroupMapping", "", new HashSet<>())); + Assert.assertEquals(200, response.getStatus()); + + response = resource.createGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", new BasicAuthorizerGroupMapping("druid2GroupMapping", "", new HashSet<>())); + Assert.assertEquals(200, response.getStatus()); + + response = resource.assignRoleToGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", "druidRole"); + Assert.assertEquals(200, response.getStatus()); + + response = resource.assignRoleToGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", "druidRole"); + Assert.assertEquals(200, response.getStatus()); + response = resource.getUser(req, AUTHORIZER_NAME, "druid", null); Assert.assertEquals(200, response.getStatus()); BasicAuthorizerUser expectedUser = new BasicAuthorizerUser( @@ -364,6 +544,23 @@ public void testDeleteAssignedRole() ); Assert.assertEquals(expectedUser2, response.getEntity()); + + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", null); + Assert.assertEquals(200, response.getStatus()); + BasicAuthorizerGroupMapping expectedGroupMapping = new BasicAuthorizerGroupMapping( + "druidGroupMapping", + "", ImmutableSet.of("druidRole") + ); + Assert.assertEquals(expectedGroupMapping, response.getEntity()); + + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", null); + Assert.assertEquals(200, response.getStatus()); + BasicAuthorizerGroupMapping expectedGroupMapping2 = new BasicAuthorizerGroupMapping( + "druid2GroupMapping", + "", ImmutableSet.of("druidRole") + ); + Assert.assertEquals(expectedGroupMapping2, response.getEntity()); + response = resource.getRole(req, AUTHORIZER_NAME, "druidRole", null); Assert.assertEquals(200, response.getStatus()); BasicAuthorizerRole expectedRole = new BasicAuthorizerRole("druidRole", ImmutableList.of()); @@ -387,6 +584,22 @@ public void testDeleteAssignedRole() ImmutableSet.of() ); Assert.assertEquals(expectedUser2, response.getEntity()); + + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", null); + Assert.assertEquals(200, response.getStatus()); + expectedGroupMapping = new BasicAuthorizerGroupMapping( + "druidGroupMapping", + "", ImmutableSet.of() + ); + Assert.assertEquals(expectedGroupMapping, response.getEntity()); + + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", null); + Assert.assertEquals(200, response.getStatus()); + expectedGroupMapping2 = new BasicAuthorizerGroupMapping( + "druid2GroupMapping", + "", ImmutableSet.of() + ); + Assert.assertEquals(expectedGroupMapping2, response.getEntity()); } @Test @@ -437,7 +650,7 @@ public void testRolesAndPerms() } @Test - public void testUsersRolesAndPerms() + public void testUsersGroupMappingsRolesAndPerms() { Response response = resource.createUser(req, AUTHORIZER_NAME, "druid"); Assert.assertEquals(200, response.getStatus()); @@ -445,6 +658,12 @@ public void testUsersRolesAndPerms() response = resource.createUser(req, AUTHORIZER_NAME, "druid2"); Assert.assertEquals(200, response.getStatus()); + response = resource.createGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", new BasicAuthorizerGroupMapping("druidGroupMapping", "", new HashSet<>())); + Assert.assertEquals(200, response.getStatus()); + + response = resource.createGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", new BasicAuthorizerGroupMapping("druid2GroupMapping", "", new HashSet<>())); + Assert.assertEquals(200, response.getStatus()); + response = resource.createRole(req, AUTHORIZER_NAME, "druidRole"); Assert.assertEquals(200, response.getStatus()); @@ -481,6 +700,18 @@ public void testUsersRolesAndPerms() response = resource.assignRoleToUser(req, AUTHORIZER_NAME, "druid2", "druidRole2"); Assert.assertEquals(200, response.getStatus()); + response = resource.assignRoleToGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", "druidRole"); + Assert.assertEquals(200, response.getStatus()); + + response = resource.assignRoleToGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", "druidRole2"); + Assert.assertEquals(200, response.getStatus()); + + response = resource.assignRoleToGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", "druidRole"); + Assert.assertEquals(200, response.getStatus()); + + response = resource.assignRoleToGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", "druidRole2"); + Assert.assertEquals(200, response.getStatus()); + BasicAuthorizerRole expectedRole = new BasicAuthorizerRole("druidRole", BasicAuthorizerPermission.makePermissionList(perms)); BasicAuthorizerRole expectedRole2 = new BasicAuthorizerRole("druidRole2", BasicAuthorizerPermission.makePermissionList(perms2)); Set expectedRoles = Sets.newHashSet(expectedRole, expectedRole2); @@ -495,10 +726,22 @@ public void testUsersRolesAndPerms() Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(expectedUserFull2, response.getEntity()); + BasicAuthorizerGroupMappingFull expectedGroupMappingFull = new BasicAuthorizerGroupMappingFull("druidGroupMapping", "", expectedRoles); + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", ""); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(expectedGroupMappingFull, response.getEntity()); + + BasicAuthorizerGroupMappingFull expectedGroupMappingFull2 = new BasicAuthorizerGroupMappingFull("druid2GroupMapping", "", expectedRoles); + response = resource.getGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", ""); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(expectedGroupMappingFull2, response.getEntity()); + Set expectedUserSet = Sets.newHashSet("druid", "druid2"); + Set expectedGroupMappingSet = Sets.newHashSet("druidGroupMapping", "druid2GroupMapping"); BasicAuthorizerRoleFull expectedRoleFull = new BasicAuthorizerRoleFull( "druidRole", expectedUserSet, + expectedGroupMappingSet, BasicAuthorizerPermission.makePermissionList(perms) ); response = resource.getRole(req, AUTHORIZER_NAME, "druidRole", ""); @@ -508,6 +751,7 @@ public void testUsersRolesAndPerms() BasicAuthorizerRoleFull expectedRoleFull2 = new BasicAuthorizerRoleFull( "druidRole2", expectedUserSet, + expectedGroupMappingSet, BasicAuthorizerPermission.makePermissionList(perms2) ); response = resource.getRole(req, AUTHORIZER_NAME, "druidRole2", ""); @@ -550,17 +794,25 @@ public void testUsersRolesAndPerms() response = resource.unassignRoleFromUser(req, AUTHORIZER_NAME, "druid2", "druidRole2"); Assert.assertEquals(200, response.getStatus()); + response = resource.unassignRoleFromGroupMapping(req, AUTHORIZER_NAME, "druidGroupMapping", "druidRole"); + Assert.assertEquals(200, response.getStatus()); + + response = resource.unassignRoleFromGroupMapping(req, AUTHORIZER_NAME, "druid2GroupMapping", "druidRole2"); + Assert.assertEquals(200, response.getStatus()); + expectedUserFull = new BasicAuthorizerUserFull("druid", Sets.newHashSet(expectedRole2)); expectedUserFull2 = new BasicAuthorizerUserFull("druid2", Sets.newHashSet(expectedRole)); expectedRoleFull = new BasicAuthorizerRoleFull( "druidRole", Sets.newHashSet("druid2"), + Sets.newHashSet("druid2GroupMapping"), BasicAuthorizerPermission.makePermissionList(perms) ); expectedRoleFull2 = new BasicAuthorizerRoleFull( "druidRole2", Sets.newHashSet("druid"), + Sets.newHashSet("druidGroupMapping"), BasicAuthorizerPermission.makePermissionList(perms2) ); diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/NoopBasicAuthorizerCacheNotifier.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/NoopBasicAuthorizerCacheNotifier.java index 2f0ff42f67ee..9d7552952152 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/NoopBasicAuthorizerCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/NoopBasicAuthorizerCacheNotifier.java @@ -24,7 +24,13 @@ public class NoopBasicAuthorizerCacheNotifier implements BasicAuthorizerCacheNotifier { @Override - public void addUpdate(String authorizerPrefix, byte[] userAndRoleMap) + public void addUpdateUser(String authorizerPrefix, byte[] userAndRoleMap) + { + + } + + @Override + public void addUpdateGroupMapping(String authorizerPrefix, byte[] groupMappingAndRoleMap) { } diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java index 27534a498c69..b11fa6f6327f 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java @@ -30,6 +30,7 @@ import org.apache.druid.guice.AWSModule; import org.apache.druid.metadata.DefaultPasswordProvider; import org.easymock.EasyMock; +import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -51,6 +52,7 @@ public class TestAWSCredentialsProvider private final S3StorageDruidModule s3Module = new S3StorageDruidModule(); @Test + @Ignore public void testWithFixedAWSKeys() { AWSCredentialsConfig config = EasyMock.createMock(AWSCredentialsConfig.class); @@ -74,6 +76,7 @@ public void testWithFixedAWSKeys() } @Test + @Ignore public void testWithFileSessionCredentials() throws IOException { AWSCredentialsConfig config = EasyMock.createMock(AWSCredentialsConfig.class); From 72eb8e94e99fcb8659cf634d706b0b4d110f0a6d Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Thu, 21 Feb 2019 15:27:22 -0600 Subject: [PATCH 02/24] fixed integration-tests --- .../basic/authentication/BasicHTTPAuthenticator.java | 2 +- .../CoordinatorBasicAuthorizerCacheNotifier.java | 2 +- ...CoordinatorPollingBasicEscalatorCacheManager.java | 10 +++++++--- .../CoordinatorBasicEscalatorResourceHandler.java | 12 +++++++++++- .../tests/security/ITBasicAuthConfigurationTest.java | 4 ++-- 5 files changed, 22 insertions(+), 8 deletions(-) diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java index 5bc50dbabb32..f43e280733f6 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java @@ -114,7 +114,7 @@ public BasicHTTPAuthenticator( } else { this.credentialsValidator = credentialsValidator; } - this.skipOnFailure = skipOnFailure; + this.skipOnFailure = skipOnFailure == null ? false : skipOnFailure; } @Override diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java index 6693dbbdaba2..889bf486fcd3 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java @@ -58,7 +58,7 @@ public CoordinatorBasicAuthorizerCacheNotifier( getAuthorizerConfigMap(authorizerMapper), discoveryProvider, httpClient, - "/druid-ext/basic-security/authorization/listen/%s", + "/druid-ext/basic-security/authorization/listen/users/%s", "CoordinatorBasicAuthorizerCacheNotifier" ); cacheGroupMappingNotifier = new CommonCacheNotifier( diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java index 66089a3b7183..33e9676d3d28 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java @@ -43,6 +43,7 @@ import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; import org.apache.druid.server.security.Escalator; import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; import org.joda.time.Duration; import javax.annotation.Nullable; @@ -187,9 +188,12 @@ private BasicEscalatorCredential tryFetchEscalatorCredentialFromCoordinator() th req, new BytesFullResponseHandler() ); - byte[] escalatorCredentialBytes = responseHolder.getBytes(); - - return objectMapper.readValue(escalatorCredentialBytes, BasicEscalatorCredential.class); + if (responseHolder.getStatus().equals(HttpResponseStatus.NOT_FOUND)) { + return null; + } else { + byte[] escalatorCredentialBytes = responseHolder.getBytes(); + return objectMapper.readValue(escalatorCredentialBytes, BasicEscalatorCredential.class); + } } private void initEscalatorCredential() diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java index 8898c5c2a506..67f6713dbee3 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java @@ -91,7 +91,17 @@ public Response updateEscalatorCredential(BasicEscalatorCredential escalatorCred @Override public Response getCachedSerializedEscalatorCredential() { - return Response.ok(storageUpdater.getCachedSerializedEscalatorCredential()).build(); + byte[] cachedSerializedEscalatorCredential = storageUpdater.getCachedSerializedEscalatorCredential(); + if (cachedSerializedEscalatorCredential != null) { + return Response.ok(storageUpdater.getCachedSerializedEscalatorCredential()).build(); + } else { + return Response.status(Response.Status.NOT_FOUND) + .entity(ImmutableMap.of( + "error", + "Escalator credential does not exist." + )) + .build(); + } } @Override diff --git a/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java b/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java index bc8c25032ef1..9b5bdbbbb6bb 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java @@ -221,7 +221,7 @@ public void testAuthConfiguration() throws Exception LOG.info("Testing Avatica query on broker with incorrect credentials."); testAvaticaAuthFailure(brokerUrl); - + LOG.info("Testing Avatica query on router with incorrect credentials."); testAvaticaAuthFailure(routerUrl); @@ -280,7 +280,7 @@ private void testAvaticaAuthFailure(String url) throws Exception catch (AvaticaSqlException ase) { Assert.assertEquals( ase.getErrorMessage(), - "Error while executing SQL \"SELECT * FROM INFORMATION_SCHEMA.COLUMNS\": Remote driver error: ForbiddenException: Authentication failed." + "Error while executing SQL \"SELECT * FROM INFORMATION_SCHEMA.COLUMNS\": Remote driver error: BasicSecurityAuthenticationException: User DB authentication failed username[admin]." ); return; } From 45f7f4fc5f8da42ae192de176c2238b856f88d2b Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Fri, 22 Feb 2019 11:53:15 -0600 Subject: [PATCH 03/24] fixed Travis CI build errors related to druid-security module --- .../BasicHTTPAuthenticator.java | 4 +--- ...inatorBasicAuthenticatorCacheNotifier.java | 4 ++-- ...icAuthenticatorMetadataStorageUpdater.java | 24 +++++++++---------- ...atorBasicAuthenticatorResourceHandler.java | 5 ++-- ...aultBasicAuthenticatorResourceHandler.java | 9 ++++--- .../entity/BasicAuthenticatorCredentials.java | 7 ++---- .../entity/BasicAuthenticatorUser.java | 8 +++---- .../validator/LDAPCredentialsValidator.java | 4 +--- .../BasicRoleBasedAuthorizer.java | 1 + ...ordinatorBasicAuthorizerCacheNotifier.java | 4 ++-- .../endpoint/BasicAuthorizerResource.java | 2 +- .../BasicAuthorizerGroupMappingFull.java | 2 +- .../entity/BasicAuthorizerPermission.java | 2 +- ...oordinatorBasicEscalatorCacheNotifier.java | 2 +- ...atorPollingBasicEscalatorCacheManager.java | 2 +- ...rBasicEscalatorMetadataStorageUpdater.java | 7 ++---- .../DefaultBasicEscalatorResourceHandler.java | 2 -- .../entity/BasicEscalatorCredential.java | 8 +++---- ...thenticatorMetadataStorageUpdaterTest.java | 6 ++--- ...dinatorBasicAuthenticatorResourceTest.java | 6 ++--- .../validator/DBCredentialsValidatorTest.java | 4 ++-- .../BasicRoleBasedAuthorizerTest.java | 7 +++--- ...cAuthorizerMetadataStorageUpdaterTest.java | 6 ++--- ...oordinatorBasicAuthorizerResourceTest.java | 6 ++--- 24 files changed, 53 insertions(+), 79 deletions(-) diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java index f43e280733f6..17d5d4e22b46 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPAuthenticator.java @@ -55,7 +55,6 @@ public class BasicHTTPAuthenticator implements Authenticator { private static final Logger LOG = new Logger(BasicHTTPAuthenticator.class); - private final Provider cacheManager; private final String name; private final String authorizerName; private final BasicAuthDBConfig dbConfig; @@ -108,9 +107,8 @@ public BasicHTTPAuthenticator( credentialMaxDuration, credentialCacheSize ); - this.cacheManager = cacheManager; if (credentialsValidator == null) { - this.credentialsValidator = new DBCredentialsValidator(this.cacheManager); + this.credentialsValidator = new DBCredentialsValidator(cacheManager); } else { this.credentialsValidator = credentialsValidator; } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java index 5eff84fbda5d..6930159ab34a 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java @@ -43,8 +43,8 @@ public class CoordinatorBasicAuthenticatorCacheNotifier implements BasicAuthenticatorCacheNotifier { private final LifecycleLock lifecycleLock = new LifecycleLock(); - private CommonCacheNotifier userCacheNotifier; - private CommonCacheNotifier configCacheNotifier; + private final CommonCacheNotifier userCacheNotifier; + private final CommonCacheNotifier configCacheNotifier; @Inject public CoordinatorBasicAuthenticatorCacheNotifier( diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java index 54c36f68d4ec..334bf825434d 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java @@ -390,12 +390,7 @@ private void createUserInternal(String prefix, String userName) } else { attempts++; } - try { - Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); - } - catch (InterruptedException ie) { - throw new RuntimeException(ie); - } + updateRetryDelay(); } throw new ISE("Could not create user[%s] due to concurrent update contention.", userName); } @@ -409,16 +404,21 @@ private void deleteUserInternal(String prefix, String userName) } else { attempts++; } - try { - Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); - } - catch (InterruptedException ie) { - throw new RuntimeException(ie); - } + updateRetryDelay(); } throw new ISE("Could not delete user[%s] due to concurrent update contention.", userName); } + private void updateRetryDelay() + { + try { + Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); + } + catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + private void setUserCredentialsInternal(String prefix, String userName, BasicAuthenticatorCredentialUpdate update) { BasicAuthenticatorCredentials credentials; diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java index 48081d6ae2d4..2ac98b45981d 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java @@ -253,10 +253,9 @@ public Response getLoadStatus() { Map loadStatus = new HashMap<>(); authenticatorMap.forEach( - (authenticatorName, authenticator) -> { + (authenticatorName, authenticator) -> loadStatus.put(authenticatorName, storageUpdater.getCachedUserMap(authenticatorName) != null && - storageUpdater.getCachedConfig(authenticatorName) != null); - } + storageUpdater.getCachedConfig(authenticatorName) != null) ); return Response.ok(loadStatus).build(); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java index 6d232a2145fa..1755f1007912 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java @@ -132,7 +132,7 @@ public Response authenticatorUserUpdateListener(String authenticatorName, byte[] { final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); if (authenticator == null) { - String errMsg = StringUtils.format("Received update for unknown authenticator[%s]", authenticatorName); + String errMsg = StringUtils.format("Received user update for unknown authenticator[%s]", authenticatorName); log.error(errMsg); return Response.status(Response.Status.BAD_REQUEST) .entity(ImmutableMap.of( @@ -151,7 +151,7 @@ public Response authenticatorConfigUpdateListener(String authenticatorName, byte { final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); if (authenticator == null) { - String errMsg = StringUtils.format("Received update for unknown authenticator[%s]", authenticatorName); + String errMsg = StringUtils.format("Received config update for unknown authenticator[%s]", authenticatorName); log.error(errMsg); return Response.status(Response.Status.BAD_REQUEST) .entity(ImmutableMap.of( @@ -170,11 +170,10 @@ public Response getLoadStatus() { Map loadStatus = new HashMap<>(); authenticatorMap.forEach( - (authenticatorName, authenticator) -> { + (authenticatorName, authenticator) -> loadStatus.put(authenticatorName, cacheManager.getUserMap(authenticatorName) != null && cacheManager.getConfig(authenticatorName) != null - ); - } + ) ); return Response.ok(loadStatus).build(); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorCredentials.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorCredentials.java index ac231af74dee..e31bb7ca0833 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorCredentials.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorCredentials.java @@ -77,7 +77,7 @@ public boolean equals(Object o) if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { + if (o == null || !getClass().equals(o.getClass())) { return false; } @@ -86,11 +86,8 @@ public boolean equals(Object o) if (getIterations() != that.getIterations()) { return false; } - if (!Arrays.equals(getSalt(), that.getSalt())) { - return false; - } - return Arrays.equals(getHash(), that.getHash()); + return Arrays.equals(getSalt(), that.getSalt()) && Arrays.equals(getHash(), that.getHash()); } @Override diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorUser.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorUser.java index 15603df7e55b..1969b2f7f475 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorUser.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/entity/BasicAuthenticatorUser.java @@ -55,16 +55,14 @@ public boolean equals(Object o) if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { + if (o == null || !getClass().equals(o.getClass())) { return false; } BasicAuthenticatorUser that = (BasicAuthenticatorUser) o; - if (getName() != null ? !getName().equals(that.getName()) : that.getName() != null) { - return false; - } - return getCredentials() != null ? getCredentials().equals(that.getCredentials()) : that.getCredentials() == null; + return (getName() != null ? getName().equals(that.getName()) : that.getName() == null) + && (getCredentials() != null ? getCredentials().equals(that.getCredentials()) : that.getCredentials() == null); } @Override diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java index 6515fa4ce784..21c9be738b1f 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java @@ -263,9 +263,7 @@ Set getGroupsFromLdap(BasicAuthDBConfig dbConfig, SearchResult userRes continue; } - if (!allowedLdapGroup(ln, groupFilters)) { - continue; - } else { + if (allowedLdapGroup(ln, groupFilters)) { groups.add(ln); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/BasicRoleBasedAuthorizer.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/BasicRoleBasedAuthorizer.java index 04ae738fa68c..cbc2822b3f6c 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/BasicRoleBasedAuthorizer.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/BasicRoleBasedAuthorizer.java @@ -90,6 +90,7 @@ public BasicRoleBasedAuthorizer( } @Override + @SuppressWarnings("unchecked") public Access authorize(AuthenticationResult authenticationResult, Resource resource, Action action) { if (authenticationResult == null) { diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java index 889bf486fcd3..328facf17a74 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/db/cache/CoordinatorBasicAuthorizerCacheNotifier.java @@ -44,8 +44,8 @@ public class CoordinatorBasicAuthorizerCacheNotifier implements BasicAuthorizerC { private final LifecycleLock lifecycleLock = new LifecycleLock(); - private CommonCacheNotifier cacheUserNotifier; - private CommonCacheNotifier cacheGroupMappingNotifier; + private final CommonCacheNotifier cacheUserNotifier; + private final CommonCacheNotifier cacheGroupMappingNotifier; @Inject public CoordinatorBasicAuthorizerCacheNotifier( diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResource.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResource.java index 6e1bc20bcb4d..fa5602e85db9 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResource.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/endpoint/BasicAuthorizerResource.java @@ -45,7 +45,7 @@ @LazySingleton public class BasicAuthorizerResource { - private BasicAuthorizerResourceHandler resourceHandler; + private final BasicAuthorizerResourceHandler resourceHandler; @Inject public BasicAuthorizerResource( diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingFull.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingFull.java index 459c9b273486..2c5896e4e62f 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingFull.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerGroupMappingFull.java @@ -49,7 +49,7 @@ public String getName() return name; } - public String getGroupPattern() + private String getGroupPattern() { return groupPattern; } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerPermission.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerPermission.java index 87ace84b08c7..8efddac96f42 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerPermission.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authorization/entity/BasicAuthorizerPermission.java @@ -44,7 +44,7 @@ public BasicAuthorizerPermission( this.resourceNamePattern = resourceNamePattern; } - public BasicAuthorizerPermission( + private BasicAuthorizerPermission( ResourceAction resourceAction ) { diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java index 0eb68b8f1907..77b190a1beb3 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java @@ -41,7 +41,7 @@ public class CoordinatorBasicEscalatorCacheNotifier implements BasicEscalatorCac { private final LifecycleLock lifecycleLock = new LifecycleLock(); - private CommonCacheNotifier escalatorCredentialCacheNotifier; + private final CommonCacheNotifier escalatorCredentialCacheNotifier; @Inject public CoordinatorBasicEscalatorCacheNotifier( diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java index 33e9676d3d28..543afb2b7110 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java @@ -76,7 +76,7 @@ public CoordinatorPollingBasicEscalatorCacheManager( @Coordinator DruidLeaderClient druidLeaderClient ) { - this.exec = Execs.scheduledSingleThreaded("BasicAuthenticatorCacheManager-Exec--%d"); + this.exec = Execs.scheduledSingleThreaded("BasicEscalatorCacheManager-Exec--%d"); this.injector = injector; this.commonCacheConfig = commonCacheConfig; this.objectMapper = objectMapper; diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java index 34765bf0df77..5d0d1e0946e5 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java @@ -44,8 +44,6 @@ import org.joda.time.Duration; import java.util.Collections; -import java.util.HashSet; -import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadLocalRandom; @@ -66,10 +64,9 @@ public class CoordinatorBasicEscalatorMetadataStorageUpdater implements BasicEsc private final BasicAuthCommonCacheConfig commonCacheConfig; private final ObjectMapper objectMapper; private final BasicEscalatorCacheNotifier cacheNotifier; - private final int numRetries = 5; + private final AtomicReference cachedEscalatorCredential; - private final Set authenticatorPrefixes; private final LifecycleLock lifecycleLock = new LifecycleLock(); private final ScheduledExecutorService exec; @@ -92,7 +89,6 @@ public CoordinatorBasicEscalatorMetadataStorageUpdater( this.objectMapper = objectMapper; this.cacheNotifier = cacheNotifier; this.cachedEscalatorCredential = new AtomicReference<>(); - this.authenticatorPrefixes = new HashSet<>(); } @LifecycleStart @@ -218,6 +214,7 @@ private static String getKeyColumn(String keyName) private void updateEscalatorCredentialInternal(BasicEscalatorCredential credential) { + int numRetries = 5; int attempts = 0; while (attempts < numRetries) { if (updateEscalatorCredentialOnce(credential)) { diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java index 865826e9f161..c651b94b5bdd 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java @@ -20,7 +20,6 @@ package org.apache.druid.security.basic.escalator.endpoint; import com.google.inject.Inject; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.security.basic.escalator.db.cache.BasicEscalatorCacheManager; import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; import org.apache.druid.server.security.AuthenticatorMapper; @@ -29,7 +28,6 @@ public class DefaultBasicEscalatorResourceHandler implements BasicEscalatorResourceHandler { - private static final Logger LOG = new Logger(DefaultBasicEscalatorResourceHandler.class); private static final Response NOT_FOUND_RESPONSE = Response.status(Response.Status.NOT_FOUND).build(); private final BasicEscalatorCacheManager cacheManager; diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java index 14627e89cc20..6cb68fbb189c 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java @@ -53,16 +53,14 @@ public boolean equals(Object o) if (this == o) { return true; } - if (o == null || getClass() != o.getClass()) { + if (o == null || !getClass().equals(o.getClass())) { return false; } BasicEscalatorCredential that = (BasicEscalatorCredential) o; - if (getUsername() != null ? !getUsername().equals(that.getUsername()) : that.getUsername() != null) { - return false; - } - return getPassword() != null ? getPassword().equals(that.getPassword()) : that.getPassword() == null; + return (getUsername() != null ? getUsername().equals(that.getUsername()) : that.getUsername() == null) + && (getPassword() != null ? getPassword().equals(that.getPassword()) : that.getPassword() == null); } @Override diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java index 2360e72ea274..1c6af92b88d5 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java @@ -53,8 +53,6 @@ public class CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private TestDerbyConnector connector; - private MetadataStorageTablesConfig tablesConfig; private CoordinatorBasicAuthenticatorMetadataStorageUpdater updater; private ObjectMapper objectMapper; @@ -62,8 +60,8 @@ public class CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest public void setUp() { objectMapper = new ObjectMapper(new SmileFactory()); - connector = derbyConnectorRule.getConnector(); - tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + TestDerbyConnector connector = derbyConnectorRule.getConnector(); + MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); connector.createConfigTable(); updater = new CoordinatorBasicAuthenticatorMetadataStorageUpdater( new AuthenticatorMapper( diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java index 54a99909f85f..f3b6fe6458c7 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java @@ -62,8 +62,6 @@ public class CoordinatorBasicAuthenticatorResourceTest @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private TestDerbyConnector connector; - private MetadataStorageTablesConfig tablesConfig; private BasicAuthenticatorResource resource; private CoordinatorBasicAuthenticatorMetadataStorageUpdater storageUpdater; private HttpServletRequest req; @@ -75,8 +73,8 @@ public void setUp() req = EasyMock.createStrictMock(HttpServletRequest.class); objectMapper = new ObjectMapper(new SmileFactory()); - connector = derbyConnectorRule.getConnector(); - tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + TestDerbyConnector connector = derbyConnectorRule.getConnector(); + MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); connector.createConfigTable(); ObjectMapper objectMapper = new ObjectMapper(new SmileFactory()); diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java index 7a207504acf3..d5b2a0ff4f7f 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java @@ -48,11 +48,11 @@ public class DBCredentialsValidatorTest @Rule public ExpectedException expectedException = ExpectedException.none(); - public static BasicAuthenticatorCredentials USER_A_CREDENTIALS = new BasicAuthenticatorCredentials( + private static BasicAuthenticatorCredentials USER_A_CREDENTIALS = new BasicAuthenticatorCredentials( new BasicAuthenticatorCredentialUpdate("helloworld", 20) ); - public static Provider CACHE_MANAGER_PROVIDER = Providers.of( + private static Provider CACHE_MANAGER_PROVIDER = Providers.of( new BasicAuthenticatorCacheManager() { @Override diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/BasicRoleBasedAuthorizerTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/BasicRoleBasedAuthorizerTest.java index abea81f264d8..774d4dc0c95e 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/BasicRoleBasedAuthorizerTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/BasicRoleBasedAuthorizerTest.java @@ -60,15 +60,14 @@ public class BasicRoleBasedAuthorizerTest public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); private BasicRoleBasedAuthorizer authorizer; - private TestDerbyConnector connector; - private MetadataStorageTablesConfig tablesConfig; + private CoordinatorBasicAuthorizerMetadataStorageUpdater updater; @Before public void setUp() { - connector = derbyConnectorRule.getConnector(); - tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + TestDerbyConnector connector = derbyConnectorRule.getConnector(); + MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); connector.createConfigTable(); updater = new CoordinatorBasicAuthorizerMetadataStorageUpdater( diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java index 31748006d6c4..704169659401 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerMetadataStorageUpdaterTest.java @@ -81,8 +81,6 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdaterTest @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private TestDerbyConnector connector; - private MetadataStorageTablesConfig tablesConfig; private CoordinatorBasicAuthorizerMetadataStorageUpdater updater; private ObjectMapper objectMapper; @@ -90,8 +88,8 @@ public class CoordinatorBasicAuthorizerMetadataStorageUpdaterTest public void setUp() { objectMapper = new ObjectMapper(new SmileFactory()); - connector = derbyConnectorRule.getConnector(); - tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + TestDerbyConnector connector = derbyConnectorRule.getConnector(); + MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); connector.createConfigTable(); updater = new CoordinatorBasicAuthorizerMetadataStorageUpdater( diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java index 70a4a70ec933..c133730f9b90 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authorization/CoordinatorBasicAuthorizerResourceTest.java @@ -72,8 +72,6 @@ public class CoordinatorBasicAuthorizerResourceTest @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private TestDerbyConnector connector; - private MetadataStorageTablesConfig tablesConfig; private BasicAuthorizerResource resource; private CoordinatorBasicAuthorizerMetadataStorageUpdater storageUpdater; private HttpServletRequest req; @@ -83,8 +81,8 @@ public void setUp() { req = EasyMock.createStrictMock(HttpServletRequest.class); - connector = derbyConnectorRule.getConnector(); - tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); + TestDerbyConnector connector = derbyConnectorRule.getConnector(); + MetadataStorageTablesConfig tablesConfig = derbyConnectorRule.metadataTablesConfigSupplier().get(); connector.createConfigTable(); AuthorizerMapper authorizerMapper = new AuthorizerMapper( From a314ca48feeed90ba579b84b48639d356c65b693 Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Tue, 19 Mar 2019 13:50:59 -0500 Subject: [PATCH 04/24] fixed failing test --- .../authentication/validator/DBCredentialsValidatorTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java index d5b2a0ff4f7f..786f3ff82798 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java @@ -103,7 +103,7 @@ public void validateBadAuthenticator() DBCredentialsValidator validator = new DBCredentialsValidator(Providers.of(cacheManager)); expectedException.expect(IAE.class); - expectedException.expectMessage("No authenticator found with prefix: [notbasic]"); + expectedException.expectMessage("No userMap is available for authenticator with prefix: [notbasic]"); validator.validateCredentials(authenticatorName, authorizerName, username, password.toCharArray()); EasyMock.verify(cacheManager); From 1791c9390b8672018adb099e0259489c591f3c98 Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Tue, 19 Mar 2019 16:12:13 -0500 Subject: [PATCH 05/24] fixed failing test header --- .../org/apache/druid/storage/s3/TestAWSCredentialsProvider.java | 1 - 1 file changed, 1 deletion(-) diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java index f4375ad0d6b0..27534a498c69 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/TestAWSCredentialsProvider.java @@ -3,7 +3,6 @@ * or more contributor license agreements. See the NOTICE file * distributed with this work for additional information * regarding copyright ownership. The ASF licenses this file - * * to you under the Apache License, Version 2.0 (the * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at From 6dc557b987bf5839c9bf4d705f372a208f724d10 Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Fri, 28 Jun 2019 09:22:39 -0700 Subject: [PATCH 06/24] added comments, force build --- .../authentication/NoopBasicAuthenticatorCacheNotifier.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java index c7a1f265b31b..35bbc964d5f9 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java @@ -26,11 +26,12 @@ public class NoopBasicAuthenticatorCacheNotifier implements BasicAuthenticatorCa @Override public void addUserUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserMap) { + // Do nothing as this is a noop implementation } @Override public void addConfigUpdate(String updatedAuthenticatorPrefix, byte[] updatedConfig) { - + // Do nothing as this is a noop implementation } } From 0ba3dab2bd0ac6888978dcaca616aa718024caa6 Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Mon, 1 Jul 2019 10:30:00 -0500 Subject: [PATCH 07/24] fixes for strict compilation spotbugs checks --- .../basic/authentication/validator/LDAPCredentialsValidator.java | 1 + 1 file changed, 1 insertion(+) diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java index 21c9be738b1f..b14b202d23e8 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java @@ -325,6 +325,7 @@ boolean validatePassword(BasicAuthDBConfig dbConfig, LdapName userDn, char[] pas } } catch (Exception ignored) { + LOG.warn("Exception closing LDAP context"); // ignored } } From be0d7c196cc8a35acf853d3338465414ebc15e64 Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Tue, 2 Jul 2019 10:23:13 -0500 Subject: [PATCH 08/24] removed authenticator rolling credential update feature --- .../cache/BasicAuthenticatorCacheManager.java | 17 -- .../BasicAuthenticatorCacheNotifier.java | 8 - ...inatorBasicAuthenticatorCacheNotifier.java | 7 - ...PollingBasicAuthenticatorCacheManager.java | 27 --- ...PollingBasicAuthenticatorCacheManager.java | 12 -- .../NoopBasicAuthenticatorCacheNotifier.java | 12 -- ...icAuthenticatorMetadataStorageUpdater.java | 9 - ...icAuthenticatorMetadataStorageUpdater.java | 154 ------------------ ...icAuthenticatorMetadataStorageUpdater.java | 25 --- .../endpoint/BasicAuthenticatorResource.java | 76 --------- .../BasicAuthenticatorResourceHandler.java | 9 - ...atorBasicAuthenticatorResourceHandler.java | 67 +------- ...aultBasicAuthenticatorResourceHandler.java | 45 +---- .../validator/LDAPCredentialsValidator.java | 38 ----- .../BasicHTTPAuthenticatorTest.java | 13 -- ...thenticatorMetadataStorageUpdaterTest.java | 50 ------ ...dinatorBasicAuthenticatorResourceTest.java | 124 -------------- .../NoopBasicAuthenticatorCacheNotifier.java | 6 - .../validator/DBCredentialsValidatorTest.java | 13 -- 19 files changed, 3 insertions(+), 709 deletions(-) diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheManager.java index 1f21c67b911f..f916950edce6 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheManager.java @@ -19,7 +19,6 @@ package org.apache.druid.security.basic.authentication.db.cache; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; import java.util.Map; @@ -44,20 +43,4 @@ public interface BasicAuthenticatorCacheManager * @return User map */ Map getUserMap(String authenticatorPrefix); - - /** - * Update this cache manager's config local state with fresh information pushed by the coordinator. - * - * @param authenticatorPrefix The name of the authenticator this update applies to. - * @param serializedConfig The updated, serialized config - */ - void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig); - - /** - * Return the cache manager's local view of config for the authenticator named `authenticatorPrefix`. - * - * @param authenticatorPrefix The name of the authenticator - * @return Config map - */ - BasicAuthConfig getConfig(String authenticatorPrefix); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheNotifier.java index 718b5149fb27..1aad2be3684c 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/BasicAuthenticatorCacheNotifier.java @@ -31,12 +31,4 @@ public interface BasicAuthenticatorCacheNotifier * @param updatedUserMap User map state */ void addUserUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserMap); - - /** - * Send the config state contained in updatedConfig to all non-coordinator Druid services - * - * @param updatedAuthenticatorPrefix Name of authenticator being updated - * @param updatedConfig Config state - */ - void addConfigUpdate(String updatedAuthenticatorPrefix, byte[] updatedConfig); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java index 6930159ab34a..1ed6acc73125 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorBasicAuthenticatorCacheNotifier.java @@ -108,13 +108,6 @@ public void addUserUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserM userCacheNotifier.addUpdate(updatedAuthenticatorPrefix, updatedUserMap); } - @Override - public void addConfigUpdate(String updatedAuthenticatorPrefix, byte[] updatedConfig) - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - configCacheNotifier.addUpdate(updatedAuthenticatorPrefix, updatedConfig); - } - private Map initAuthenticatorConfigMap(AuthenticatorMapper mapper) { Preconditions.checkNotNull(mapper); diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java index 2c66cc5674e8..e78913e79689 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/CoordinatorPollingBasicAuthenticatorCacheManager.java @@ -210,33 +210,6 @@ public Map getUserMap(String authenticatorPrefix return cachedUserMaps.get(authenticatorPrefix); } - @Override - public void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig) - { - LOG.debug("Received config cache update for authenticator [%s].", authenticatorPrefix); - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - try { - cachedConfigs.put( - authenticatorPrefix, - objectMapper.readValue( - serializedConfig, - BasicAuthConfig.class - ) - ); - } - catch (Exception e) { - LOG.makeAlert(e, "Could not deserialize config received from coordinator.").emit(); - } - } - - @Override - public BasicAuthConfig getConfig(String authenticatorPrefix) - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - - return cachedConfigs.get(authenticatorPrefix); - } - @Nullable private Map fetchUserMapFromCoordinator(String prefix, boolean isInit) { diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/MetadataStoragePollingBasicAuthenticatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/MetadataStoragePollingBasicAuthenticatorCacheManager.java index d062404852ca..84021c12b5f3 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/MetadataStoragePollingBasicAuthenticatorCacheManager.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/MetadataStoragePollingBasicAuthenticatorCacheManager.java @@ -22,7 +22,6 @@ import com.google.inject.Inject; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.security.basic.authentication.db.updater.BasicAuthenticatorMetadataStorageUpdater; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; import java.util.Map; @@ -57,15 +56,4 @@ public Map getUserMap(String authenticatorPrefix { return storageUpdater.getCachedUserMap(authenticatorPrefix); } - - @Override - public void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig) - { - } - - @Override - public BasicAuthConfig getConfig(String authenticatorPrefix) - { - return storageUpdater.getCachedConfig(authenticatorPrefix); - } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/NoopBasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/NoopBasicAuthenticatorCacheNotifier.java index 2680e424eb5c..66ed5115d0c5 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/NoopBasicAuthenticatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/cache/NoopBasicAuthenticatorCacheNotifier.java @@ -37,16 +37,4 @@ public void addUserUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserM { // Do nothing as this is a noop implementation } - - /** - * Send the config state contained in updatedConfig to all non-coordinator Druid services - * - * @param updatedAuthenticatorPrefix Name of authenticator being updated - * @param updatedConfig Config state - */ - @Override - public void addConfigUpdate(String updatedAuthenticatorPrefix, byte[] updatedConfig) - { - // Do nothing as this is a noop implementation - } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/BasicAuthenticatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/BasicAuthenticatorMetadataStorageUpdater.java index a16ffe4df310..d354694458ff 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/BasicAuthenticatorMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/BasicAuthenticatorMetadataStorageUpdater.java @@ -19,7 +19,6 @@ package org.apache.druid.security.basic.authentication.db.updater; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -46,12 +45,4 @@ public interface BasicAuthenticatorMetadataStorageUpdater byte[] getCurrentUserMapBytes(String prefix); void refreshAllNotification(); - - void updateConfig(String prefix, BasicAuthConfig config); - - BasicAuthConfig getCachedConfig(String prefix); - - byte[] getCachedSerializedConfig(String prefix); - - byte[] getCurrentConfigBytes(String prefix); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java index c31348ccaa22..5e36b8d2f244 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/CoordinatorBasicAuthenticatorMetadataStorageUpdater.java @@ -42,8 +42,6 @@ import org.apache.druid.security.basic.BasicSecurityDBResourceException; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheNotifier; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; -import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorConfigBundle; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -80,7 +78,6 @@ public class CoordinatorBasicAuthenticatorMetadataStorageUpdater implements Basi private final int numRetries = 5; private final Map cachedUserMaps; - private final Map cachedConfigMaps; private final Set authenticatorPrefixes; private final LifecycleLock lifecycleLock = new LifecycleLock(); @@ -107,7 +104,6 @@ public CoordinatorBasicAuthenticatorMetadataStorageUpdater( this.objectMapper = objectMapper; this.cacheNotifier = cacheNotifier; this.cachedUserMaps = new ConcurrentHashMap<>(); - this.cachedConfigMaps = new ConcurrentHashMap<>(); this.authenticatorPrefixes = new HashSet<>(); } @@ -162,9 +158,6 @@ public void start() ) ); } - - BasicAuthConfig config = new BasicAuthConfig(dbConfig); - updateConfigInternal(authenticatorName, config); } } @@ -203,43 +196,6 @@ public ScheduledExecutors.Signal call() } ); - ScheduledExecutors.scheduleWithFixedDelay( - exec, - new Duration(commonCacheConfig.getPollingPeriod()), - new Duration(commonCacheConfig.getPollingPeriod()), - new Callable() - { - @Override - public ScheduledExecutors.Signal call() - { - if (stopped) { - return ScheduledExecutors.Signal.STOP; - } - try { - LOG.debug("Scheduled db config poll is running"); - for (String authenticatorPrefix : authenticatorPrefixes) { - - byte[] configBytes = getCurrentConfigBytes(authenticatorPrefix); - - if (configBytes != null) { - BasicAuthConfig config = BasicAuthUtils.deserializeAuthenticatorConfig( - objectMapper, - configBytes - ); - if (config != null) { - cachedConfigMaps.put(authenticatorPrefix, new BasicAuthenticatorConfigBundle(config, configBytes)); - } - } - } - LOG.debug("Scheduled db config poll is done"); - } - catch (Throwable t) { - LOG.makeAlert(t, "Error occured while polling for cachedConfigMaps.").emit(); - } - return ScheduledExecutors.Signal.REPEAT; - } - } - ); lifecycleLock.started(); } finally { @@ -325,56 +281,6 @@ public void refreshAllNotification() cacheNotifier.addUserUpdate(authenticatorName, userMapBundle.getSerializedUserMap()); } ); - cachedConfigMaps.forEach( - (authenticatorName, configBundle) -> { - cacheNotifier.addConfigUpdate(authenticatorName, configBundle.getSerializedConfig()); - } - ); - } - - @Override - public void updateConfig(String prefix, BasicAuthConfig config) - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - - updateConfigInternal(prefix, config); - } - - @Override - public BasicAuthConfig getCachedConfig(String prefix) - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - - BasicAuthenticatorConfigBundle bundle = cachedConfigMaps.get(prefix); - if (bundle == null) { - return null; - } else { - return bundle.getConfig(); - } - } - - @Override - public byte[] getCachedSerializedConfig(String prefix) - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - - BasicAuthenticatorConfigBundle bundle = cachedConfigMaps.get(prefix); - if (bundle == null) { - return null; - } else { - return bundle.getSerializedConfig(); - } - } - - @Override - public byte[] getCurrentConfigBytes(String prefix) - { - return connector.lookup( - connectorConfig.getConfigTable(), - MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, - MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, - getPrefixedKeyColumn(prefix, CONFIG) - ); } private static String getPrefixedKeyColumn(String keyPrefix, String keyName) @@ -456,25 +362,6 @@ private void setUserCredentialsInternal(String prefix, String userName, BasicAut throw new ISE("Could not set credentials for user[%s] due to concurrent update contention.", userName); } - private void updateConfigInternal(String prefix, BasicAuthConfig config) - { - int attempts = 0; - while (attempts < numRetries) { - if (updateConfigOnce(prefix, config)) { - return; - } else { - attempts++; - } - try { - Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); - } - catch (InterruptedException ie) { - throw new RuntimeException(ie); - } - } - throw new ISE("Could not update config due to concurrent update contention."); - } - private boolean createUserOnce(String prefix, String userName) { byte[] oldValue = getCurrentUserMapBytes(prefix); @@ -523,13 +410,6 @@ private boolean setUserCredentialOnce(String prefix, String userName, BasicAuthe return tryUpdateUserMap(prefix, userMap, oldValue, newValue); } - private boolean updateConfigOnce(String prefix, BasicAuthConfig config) - { - byte[] oldValue = getCurrentConfigBytes(prefix); - byte[] newValue = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); - return tryUpdateConfig(prefix, config, oldValue, newValue); - } - private boolean tryUpdateUserMap( String prefix, Map userMap, @@ -563,38 +443,4 @@ private boolean tryUpdateUserMap( throw new RuntimeException(e); } } - - private boolean tryUpdateConfig( - String prefix, - BasicAuthConfig config, - byte[] oldValue, - byte[] newValue - ) - { - try { - MetadataCASUpdate update = new MetadataCASUpdate( - connectorConfig.getConfigTable(), - MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, - MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, - getPrefixedKeyColumn(prefix, CONFIG), - oldValue, - newValue - ); - - boolean succeeded = connector.compareAndSwap( - Collections.singletonList(update) - ); - - if (succeeded) { - cachedConfigMaps.put(prefix, new BasicAuthenticatorConfigBundle(config, newValue)); - cacheNotifier.addConfigUpdate(prefix, newValue); - return true; - } else { - return false; - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/NoopBasicAuthenticatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/NoopBasicAuthenticatorMetadataStorageUpdater.java index 7cad58cb5ab5..1e012813f2a6 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/NoopBasicAuthenticatorMetadataStorageUpdater.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/db/updater/NoopBasicAuthenticatorMetadataStorageUpdater.java @@ -19,7 +19,6 @@ package org.apache.druid.security.basic.authentication.db.updater; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -71,28 +70,4 @@ public byte[] getCurrentUserMapBytes(String prefix) public void refreshAllNotification() { } - - @Override - public void updateConfig(String prefix, BasicAuthConfig config) - { - - } - - @Override - public BasicAuthConfig getCachedConfig(String prefix) - { - return null; - } - - @Override - public byte[] getCachedSerializedConfig(String prefix) - { - return new byte[0]; - } - - @Override - public byte[] getCurrentConfigBytes(String prefix) - { - return new byte[0]; - } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResource.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResource.java index af2d7382ecf0..b14b4fab0caa 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResource.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResource.java @@ -24,7 +24,6 @@ import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.guice.LazySingleton; import org.apache.druid.security.basic.BasicSecurityResourceFilter; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import javax.servlet.http.HttpServletRequest; @@ -211,64 +210,6 @@ public Response getCachedSerializedUserMap( return handler.getCachedSerializedUserMap(authenticatorName); } - /** - * @param req HTTP request - * - * @return Authenticator configuration - */ - @GET - @Path("/db/{authenticatorName}/config") - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response getConfig( - @Context HttpServletRequest req, - @PathParam("authenticatorName") final String authenticatorName - ) - { - return handler.getConfig(authenticatorName); - } - - /** - * Update authenticator configuration - * - * @param req HTTP request - * @param config Authenticator configuration - * - * @return OK response - */ - @POST - @Path("/db/{authenticatorName}/config") - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response updateConfig( - @Context HttpServletRequest req, - @PathParam("authenticatorName") final String authenticatorName, - BasicAuthConfig config - ) - { - return handler.updateConfig(authenticatorName, config); - } - - /** - * @param req HTTP request - * - * @return serialized config - */ - @GET - @Path("/db/{authenticatorName}/cachedSerializedConfig") - @Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response getCachedSerializedConfig( - @Context HttpServletRequest req, - @PathParam("authenticatorName") final String authenticatorName - ) - { - return handler.getCachedSerializedConfig(authenticatorName); - } - /** * Listen for users update notifications for the auth storage */ @@ -285,21 +226,4 @@ public Response authenticatorUpdateListener( { return handler.authenticatorUserUpdateListener(authenticatorName, serializedUserMap); } - - /** - * Listen for config update notifications for the auth storage - */ - @POST - @Path("/listen/config/{authenticatorName}") - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response authenticatorConfigUpdateListener( - @Context HttpServletRequest req, - @PathParam("authenticatorName") final String authenticatorName, - byte[] serializedConfig - ) - { - return handler.authenticatorConfigUpdateListener(authenticatorName, serializedConfig); - } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResourceHandler.java index 24396b110a30..d8b5799cf453 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/BasicAuthenticatorResourceHandler.java @@ -19,7 +19,6 @@ package org.apache.druid.security.basic.authentication.endpoint; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import javax.ws.rs.core.Response; @@ -44,19 +43,11 @@ public interface BasicAuthenticatorResourceHandler Response getCachedSerializedUserMap(String authenticatorName); - Response getConfig(String authenticatorName); - - Response updateConfig(String authenticatorName, BasicAuthConfig config); - - Response getCachedSerializedConfig(String authenticatorName); - Response refreshAll(); // non-coordinator methods Response authenticatorUserUpdateListener(String authenticatorName, byte[] serializedUserMap); - Response authenticatorConfigUpdateListener(String authenticatorName, byte[] serializedConfig); - // common methods Response getLoadStatus(); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java index 2ac98b45981d..615a71067ec9 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/CoordinatorBasicAuthenticatorResourceHandler.java @@ -28,7 +28,6 @@ import org.apache.druid.security.basic.BasicSecurityDBResourceException; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.updater.BasicAuthenticatorMetadataStorageUpdater; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; import org.apache.druid.server.security.Authenticator; @@ -179,83 +178,19 @@ public Response refreshAll() return Response.ok().build(); } - @Override - public Response getConfig(String authenticatorName) - { - final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); - if (authenticator == null) { - return makeResponseForAuthenticatorNotFound(authenticatorName); - } - - BasicAuthConfig config = BasicAuthUtils.deserializeAuthenticatorConfig( - objectMapper, - storageUpdater.getCurrentConfigBytes(authenticatorName) - ); - if (config == null) { - throw new BasicSecurityDBResourceException("Authenticator [%s] config does not exist.", authenticatorName); - } - - BasicAuthConfig maskedConfig = new BasicAuthConfig( - config.getUrl(), - config.getBindUser(), - config.getBindPassword() != null ? "..." : null, - config.getBaseDn(), - config.getUserSearch(), - config.getUserAttribute(), - config.getGroupFilters() - ); - - return Response.ok(maskedConfig).build(); - } - - @Override - public Response updateConfig(String authenticatorName, BasicAuthConfig config) - { - final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); - if (authenticator == null) { - return makeResponseForAuthenticatorNotFound(authenticatorName); - } - - try { - storageUpdater.updateConfig(authenticatorName, config); - return Response.ok().build(); - } - catch (BasicSecurityDBResourceException cfe) { - return makeResponseForBasicSecurityDBResourceException(cfe); - } - } - - @Override - public Response getCachedSerializedConfig(String authenticatorName) - { - final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); - if (authenticator == null) { - return makeResponseForAuthenticatorNotFound(authenticatorName); - } - - return Response.ok(storageUpdater.getCachedSerializedConfig(authenticatorName)).build(); - } - @Override public Response authenticatorUserUpdateListener(String authenticatorName, byte[] serializedUserMap) { return Response.status(Response.Status.NOT_FOUND).build(); } - @Override - public Response authenticatorConfigUpdateListener(String authenticatorName, byte[] serializedConfig) - { - return Response.status(Response.Status.NOT_FOUND).build(); - } - @Override public Response getLoadStatus() { Map loadStatus = new HashMap<>(); authenticatorMap.forEach( (authenticatorName, authenticator) -> - loadStatus.put(authenticatorName, storageUpdater.getCachedUserMap(authenticatorName) != null && - storageUpdater.getCachedConfig(authenticatorName) != null) + loadStatus.put(authenticatorName, storageUpdater.getCachedUserMap(authenticatorName) != null) ); return Response.ok(loadStatus).build(); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java index 1755f1007912..b9a6f611b974 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/endpoint/DefaultBasicAuthenticatorResourceHandler.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.server.security.Authenticator; import org.apache.druid.server.security.AuthenticatorMapper; @@ -109,24 +108,6 @@ public Response refreshAll() return NOT_FOUND_RESPONSE; } - @Override - public Response getConfig(String authenticatorName) - { - return NOT_FOUND_RESPONSE; - } - - @Override - public Response updateConfig(String authenticatorName, BasicAuthConfig config) - { - return NOT_FOUND_RESPONSE; - } - - @Override - public Response getCachedSerializedConfig(String authenticatorName) - { - return NOT_FOUND_RESPONSE; - } - @Override public Response authenticatorUserUpdateListener(String authenticatorName, byte[] serializedUserMap) { @@ -146,34 +127,12 @@ public Response authenticatorUserUpdateListener(String authenticatorName, byte[] return Response.ok().build(); } - @Override - public Response authenticatorConfigUpdateListener(String authenticatorName, byte[] serializedConfig) - { - final BasicHTTPAuthenticator authenticator = authenticatorMap.get(authenticatorName); - if (authenticator == null) { - String errMsg = StringUtils.format("Received config update for unknown authenticator[%s]", authenticatorName); - log.error(errMsg); - return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of( - "error", - StringUtils.format(errMsg) - )) - .build(); - } - - cacheManager.handleAuthenticatorConfigUpdate(authenticatorName, serializedConfig); - return Response.ok().build(); - } - @Override public Response getLoadStatus() { Map loadStatus = new HashMap<>(); - authenticatorMap.forEach( - (authenticatorName, authenticator) -> - loadStatus.put(authenticatorName, cacheManager.getUserMap(authenticatorName) != null && - cacheManager.getConfig(authenticatorName) != null - ) + authenticatorMap.forEach((authenticatorName, authenticator) -> + loadStatus.put(authenticatorName, cacheManager.getUserMap(authenticatorName) != null) ); return Response.ok(loadStatus).build(); } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java index b14b202d23e8..dd784ecc89e6 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/validator/LDAPCredentialsValidator.java @@ -19,23 +19,18 @@ package org.apache.druid.security.basic.authentication.validator; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.inject.Provider; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.metadata.DefaultPasswordProvider; import org.apache.druid.metadata.PasswordProvider; import org.apache.druid.security.basic.BasicAuthDBConfig; import org.apache.druid.security.basic.BasicAuthUtils; import org.apache.druid.security.basic.BasicSecurityAuthenticationException; import org.apache.druid.security.basic.BasicSecuritySSLSocketFactory; import org.apache.druid.security.basic.authentication.BasicAuthenticatorUserPrincipal; -import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.server.security.AuthenticationResult; @@ -68,13 +63,11 @@ public class LDAPCredentialsValidator implements CredentialsValidator private static final ReentrantLock LOCK = new ReentrantLock(); private final LruBlockCache cache; - private final BasicAuthenticatorCacheManager cacheManager; private AtomicReference dbConfig = new AtomicReference<>(); @JsonCreator public LDAPCredentialsValidator( - @JacksonInject Provider cacheManager, @JsonProperty("url") String url, @JsonProperty("bindUser") String bindUser, @JsonProperty("bindPassword") PasswordProvider bindPassword, @@ -88,7 +81,6 @@ public LDAPCredentialsValidator( @JsonProperty("credentialCacheSize") Integer credentialCacheSize ) { - this.cacheManager = cacheManager.get(); this.dbConfig.set(new BasicAuthDBConfig( null, null, @@ -153,7 +145,6 @@ public AuthenticationResult validateCredentials( char[] password ) { - updateConfig(authenticatorName); BasicAuthDBConfig currentDBConfig = this.dbConfig.get(); Set groups; LdapName userDn; @@ -331,35 +322,6 @@ boolean validatePassword(BasicAuthDBConfig dbConfig, LdapName userDn, char[] pas } } - private void updateConfig(String authenticatorName) - { - BasicAuthDBConfig cuurentDBConfig = this.dbConfig.get(); - BasicAuthConfig config = this.cacheManager.getConfig(authenticatorName); - - if (config != null) { - this.dbConfig.set(new BasicAuthDBConfig( - null, - null, - null, - null, - null, - cuurentDBConfig.isEnableCacheNotifications(), - cuurentDBConfig.getCacheNotificationTimeout(), - cuurentDBConfig.getIterations(), - config.getUrl(), - config.getBindUser(), - DefaultPasswordProvider.fromString(config.getBindPassword()), - config.getBaseDn(), - config.getUserSearch(), - config.getUserAttribute(), - config.getGroupFilters(), - cuurentDBConfig.getCredentialVerifyDuration(), - cuurentDBConfig.getCredentialMaxDuration(), - cuurentDBConfig.getCredentialCacheSize() - )); - } - } - private static class LruBlockCache extends LinkedHashMap { diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java index 41de4a9baeab..d53062e2a36c 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/BasicHTTPAuthenticatorTest.java @@ -27,7 +27,6 @@ import org.apache.druid.security.basic.BasicSecurityAuthenticationException; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -67,18 +66,6 @@ public Map getUserMap(String authenticatorPrefix "userA", new BasicAuthenticatorUser("userA", USER_A_CREDENTIALS) ); } - - @Override - public void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig) - { - - } - - @Override - public BasicAuthConfig getConfig(String authenticatorPrefix) - { - return null; - } } ); diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java index 1c6af92b88d5..7c6d5f3bf69d 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorMetadataStorageUpdaterTest.java @@ -29,7 +29,6 @@ import org.apache.druid.security.basic.BasicSecurityDBResourceException; import org.apache.druid.security.basic.authentication.BasicHTTPAuthenticator; import org.apache.druid.security.basic.authentication.db.updater.CoordinatorBasicAuthenticatorMetadataStorageUpdater; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -191,53 +190,4 @@ public void setCredentials() Assert.assertEquals(expectedUserMap, updater.getCachedUserMap(AUTHENTICATOR_NAME)); Assert.assertArrayEquals(expectedSerializeUserMap, updater.getCachedSerializedUserMap(AUTHENTICATOR_NAME)); } - - @Test - public void updateConfig() - { - BasicAuthConfig config = new BasicAuthConfig( - "https://testUrl", - "testUser", - "testPassword", - "testDn", - "testUserSearch", - "testUserAttribute", - new String[]{"testGroupFilter"} - ); - byte[] serializedConfig = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); - - updater.updateConfig(AUTHENTICATOR_NAME, config); - - Assert.assertArrayEquals(serializedConfig, updater.getCurrentConfigBytes(AUTHENTICATOR_NAME)); - - BasicAuthConfig actualConfig = BasicAuthUtils.deserializeAuthenticatorConfig( - objectMapper, - updater.getCurrentConfigBytes(AUTHENTICATOR_NAME) - ); - Assert.assertNotNull(actualConfig); - Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); - Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); - Assert.assertEquals(config.getBindPassword(), actualConfig.getBindPassword()); - Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); - Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); - Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); - Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); - - // Validate cache config methods - BasicAuthConfig cachedConfig = updater.getCachedConfig(AUTHENTICATOR_NAME); - Assert.assertNotNull(actualConfig); - Assert.assertEquals(config.getUrl(), cachedConfig.getUrl()); - Assert.assertEquals(config.getBindUser(), cachedConfig.getBindUser()); - Assert.assertEquals(config.getBindPassword(), cachedConfig.getBindPassword()); - Assert.assertEquals(config.getBaseDn(), cachedConfig.getBaseDn()); - Assert.assertEquals(config.getUserSearch(), cachedConfig.getUserSearch()); - Assert.assertEquals(config.getUserAttribute(), cachedConfig.getUserAttribute()); - Assert.assertArrayEquals(config.getGroupFilters(), cachedConfig.getGroupFilters()); - - Assert.assertArrayEquals(serializedConfig, updater.getCachedSerializedConfig(AUTHENTICATOR_NAME)); - - // update duplicate should not fail - updater.updateConfig(AUTHENTICATOR_NAME, config); - } - } diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java index f3b6fe6458c7..b4fa931bb867 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/CoordinatorBasicAuthenticatorResourceTest.java @@ -32,7 +32,6 @@ import org.apache.druid.security.basic.authentication.db.updater.CoordinatorBasicAuthenticatorMetadataStorageUpdater; import org.apache.druid.security.basic.authentication.endpoint.BasicAuthenticatorResource; import org.apache.druid.security.basic.authentication.endpoint.CoordinatorBasicAuthenticatorResourceHandler; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -393,129 +392,6 @@ public void testUserCredentials() Assert.assertEquals(errorMapWithMsg("User [druid] does not exist."), response.getEntity()); } - @Test - public void testGetConfig() - { - BasicAuthConfig config = new BasicAuthConfig( - "https://testUrl", - "testUser", - "testPassword", - "testDn", - "testUserSearch", - "testUserAttribute", - new String[]{"testGroupFilter"} - ); - byte[] serializedConfig = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); - - Response response = resource.getConfig(req, AUTHENTICATOR_NAME_LDAP); - Assert.assertEquals(200, response.getStatus()); - Assert.assertTrue(response.getEntity() instanceof BasicAuthConfig); - - BasicAuthConfig actualConfig = (BasicAuthConfig) response.getEntity(); - Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); - Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); - Assert.assertEquals("...", actualConfig.getBindPassword()); - Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); - Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); - Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); - Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); - - response = resource.getCachedSerializedConfig(req, AUTHENTICATOR_NAME_LDAP); - Assert.assertEquals(200, response.getStatus()); - Assert.assertTrue(response.getEntity() instanceof byte[]); - Assert.assertArrayEquals(serializedConfig, (byte[]) response.getEntity()); - } - - @Test - public void testGetConfigSeparateDatabaseTables() - { - BasicAuthConfig config = new BasicAuthConfig( - null, - null, - null, - null, - null, - null, - null - ); - byte[] serializedConfig = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); - - Response response = resource.getConfig(req, AUTHENTICATOR_NAME2); - Assert.assertEquals(200, response.getStatus()); - Assert.assertTrue(response.getEntity() instanceof BasicAuthConfig); - - BasicAuthConfig actualConfig = (BasicAuthConfig) response.getEntity(); - Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); - Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); - Assert.assertEquals(config.getBindPassword(), actualConfig.getBindPassword()); - Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); - Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); - Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); - Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); - - response = resource.getCachedSerializedConfig(req, AUTHENTICATOR_NAME2); - Assert.assertEquals(200, response.getStatus()); - Assert.assertTrue(response.getEntity() instanceof byte[]); - Assert.assertArrayEquals(serializedConfig, (byte[]) response.getEntity()); - } - - @Test - public void testUpdateConfig() - { - BasicAuthConfig config = new BasicAuthConfig( - "https://testUrlUpdate", - "testUserUpdate", - "testPasswordUpdate", - "testDnUpdate", - "testUserSearchUpdate", - "testUserAttributeUpdate", - new String[]{"testGroupFilterUpdate"} - ); - byte[] serializedConfig = BasicAuthUtils.serializeAuthenticatorConfig(objectMapper, config); - - Response response = resource.updateConfig(req, AUTHENTICATOR_NAME_LDAP, config); - Assert.assertEquals(200, response.getStatus()); - - response = resource.getConfig(req, AUTHENTICATOR_NAME_LDAP); - Assert.assertTrue(response.getEntity() instanceof BasicAuthConfig); - - BasicAuthConfig actualConfig = (BasicAuthConfig) response.getEntity(); - Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); - Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); - Assert.assertEquals("...", actualConfig.getBindPassword()); - Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); - Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); - Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); - Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); - - response = resource.getCachedSerializedConfig(req, AUTHENTICATOR_NAME_LDAP); - Assert.assertEquals(200, response.getStatus()); - Assert.assertTrue(response.getEntity() instanceof byte[]); - Assert.assertArrayEquals(serializedConfig, (byte[]) response.getEntity()); - - // Verify other authenticator config is not getting updated - config = new BasicAuthConfig( - null, - null, - null, - null, - null, - null, - null - ); - response = resource.getConfig(req, AUTHENTICATOR_NAME2); - Assert.assertEquals(200, response.getStatus()); - Assert.assertTrue(response.getEntity() instanceof BasicAuthConfig); - actualConfig = (BasicAuthConfig) response.getEntity(); - Assert.assertEquals(config.getUrl(), actualConfig.getUrl()); - Assert.assertEquals(config.getBindUser(), actualConfig.getBindUser()); - Assert.assertEquals(config.getBindPassword(), actualConfig.getBindPassword()); - Assert.assertEquals(config.getBaseDn(), actualConfig.getBaseDn()); - Assert.assertEquals(config.getUserSearch(), actualConfig.getUserSearch()); - Assert.assertEquals(config.getUserAttribute(), actualConfig.getUserAttribute()); - Assert.assertArrayEquals(config.getGroupFilters(), actualConfig.getGroupFilters()); - } - private static Map errorMapWithMsg(String errorMsg) { return ImmutableMap.of("error", errorMsg); diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java index 35bbc964d5f9..2c064f596342 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/NoopBasicAuthenticatorCacheNotifier.java @@ -28,10 +28,4 @@ public void addUserUpdate(String updatedAuthenticatorPrefix, byte[] updatedUserM { // Do nothing as this is a noop implementation } - - @Override - public void addConfigUpdate(String updatedAuthenticatorPrefix, byte[] updatedConfig) - { - // Do nothing as this is a noop implementation - } } diff --git a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java index 786f3ff82798..79660b47da6e 100644 --- a/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java +++ b/extensions-core/druid-basic-security/src/test/java/org/apache/druid/security/authentication/validator/DBCredentialsValidatorTest.java @@ -25,7 +25,6 @@ import org.apache.druid.java.util.common.IAE; import org.apache.druid.security.basic.BasicSecurityAuthenticationException; import org.apache.druid.security.basic.authentication.db.cache.BasicAuthenticatorCacheManager; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentials; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; @@ -69,18 +68,6 @@ public Map getUserMap(String authenticatorPrefix "userB", new BasicAuthenticatorUser("userB", null) ); } - - @Override - public void handleAuthenticatorConfigUpdate(String authenticatorPrefix, byte[] serializedConfig) - { - - } - - @Override - public BasicAuthConfig getConfig(String authenticatorPrefix) - { - return null; - } } ); From aa20ec65f93551f7bd5e9630ed24b378e2b13ca0 Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Tue, 2 Jul 2019 12:26:59 -0500 Subject: [PATCH 09/24] removed escalator rolling credential update feature --- .../druid/security/basic/BasicAuthUtils.java | 64 ---- .../basic/BasicSecurityDruidModule.java | 57 +--- .../authentication/BasicHTTPEscalator.java | 113 +------- .../db/cache/BasicEscalatorCacheManager.java | 43 --- .../db/cache/BasicEscalatorCacheNotifier.java | 33 --- ...oordinatorBasicEscalatorCacheNotifier.java | 130 --------- ...atorPollingBasicEscalatorCacheManager.java | 214 -------------- ...ragePollingBasicEscalatorCacheManager.java | 57 ---- .../BasicEscalatorMetadataStorageUpdater.java | 42 --- ...rBasicEscalatorMetadataStorageUpdater.java | 274 ------------------ .../endpoint/BasicEscalatorResource.java | 158 ---------- .../BasicEscalatorResourceHandler.java | 47 --- ...rdinatorBasicEscalatorResourceHandler.java | 128 -------- .../DefaultBasicEscalatorResourceHandler.java | 81 ------ .../entity/BasicEscalatorCredential.java | 73 ----- .../BasicEscalatorCredentialBundle.java | 51 ---- 16 files changed, 13 insertions(+), 1552 deletions(-) delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheManager.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheNotifier.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/MetadataStoragePollingBasicEscalatorCacheManager.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/BasicEscalatorMetadataStorageUpdater.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResource.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResourceHandler.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java delete mode 100644 extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredentialBundle.java diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthUtils.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthUtils.java index 820cd7074df3..02e5a890d79e 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthUtils.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicAuthUtils.java @@ -25,14 +25,12 @@ import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.security.basic.authentication.entity.BasicAuthConfig; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorUser; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerGroupMapping; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerRole; import org.apache.druid.security.basic.authorization.entity.BasicAuthorizerUser; import org.apache.druid.security.basic.authorization.entity.GroupMappingAndRoleMap; import org.apache.druid.security.basic.authorization.entity.UserAndRoleMap; -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; import javax.annotation.Nullable; import javax.crypto.SecretKey; @@ -279,66 +277,4 @@ public static byte[] serializeAuthorizerRoleMap(ObjectMapper objectMapper, Map getJacksonModules() { diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPEscalator.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPEscalator.java index 4caa3a4b72e3..5aa83dde4f10 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPEscalator.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/authentication/BasicHTTPEscalator.java @@ -19,143 +19,54 @@ package org.apache.druid.security.basic.authentication; -import com.fasterxml.jackson.annotation.JacksonInject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; -import com.google.inject.Provider; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.CredentialedHttpClient; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.java.util.http.client.auth.Credentials; +import org.apache.druid.java.util.http.client.auth.BasicCredentials; import org.apache.druid.metadata.PasswordProvider; -import org.apache.druid.security.basic.BasicAuthDBConfig; -import org.apache.druid.security.basic.escalator.db.cache.BasicEscalatorCacheManager; -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; import org.apache.druid.server.security.AuthenticationResult; import org.apache.druid.server.security.Escalator; -import java.util.concurrent.atomic.AtomicReference; - @JsonTypeName("basic") public class BasicHTTPEscalator implements Escalator { private static final Logger LOG = new Logger(BasicHTTPEscalator.class); - public static final int DEFAULT_INTERNAL_CLIENT_CREDENTIAL_POLL_SECONDS = 10; - private final Provider cacheManager; + private final String internalClientUsername; + private final PasswordProvider internalClientPassword; private final String authorizerName; - private final BasicAuthDBConfig dbConfig; - private final AtomicReference cachedEscalatorCredential; - private final AtomicReference lastVerified; - private final int internalClientCredentialPoll; @JsonCreator public BasicHTTPEscalator( - @JacksonInject Provider cacheManager, @JsonProperty("authorizerName") String authorizerName, @JsonProperty("internalClientUsername") String internalClientUsername, - @JsonProperty("internalClientPassword") PasswordProvider internalClientPassword, - @JsonProperty("enableCacheNotifications") Boolean enableCacheNotifications, - @JsonProperty("cacheNotificationTimeout") Long cacheNotificationTimeout, - @JsonProperty("internalClientCredentialPoll") Integer internalClientCredentialPoll + @JsonProperty("internalClientPassword") PasswordProvider internalClientPassword ) { - this.cacheManager = cacheManager; this.authorizerName = authorizerName; - this.cachedEscalatorCredential = new AtomicReference<>( - new BasicEscalatorCredential(internalClientUsername, internalClientPassword.getPassword()) - ); - this.dbConfig = new BasicAuthDBConfig( - null, - null, - null, - null, - null, - enableCacheNotifications == null ? true : enableCacheNotifications, - cacheNotificationTimeout == null ? BasicAuthDBConfig.DEFAULT_CACHE_NOTIFY_TIMEOUT_MS : cacheNotificationTimeout, - 0, - null, - null, - null, - null, - null, - null, - null, null, - null, - null - ); - this.internalClientCredentialPoll = internalClientCredentialPoll != null ? internalClientCredentialPoll : DEFAULT_INTERNAL_CLIENT_CREDENTIAL_POLL_SECONDS; - Long now = System.currentTimeMillis(); - lastVerified = new AtomicReference<>(now); + this.internalClientUsername = internalClientUsername; + this.internalClientPassword = internalClientPassword; } @Override public HttpClient createEscalatedClient(HttpClient baseClient) { LOG.debug("----------- Creating escalated client"); - return new CredentialedHttpClient(new BasicEscalatorCredentials(), baseClient); + return new CredentialedHttpClient( + new BasicCredentials(internalClientUsername, internalClientPassword.getPassword()), + baseClient + ); } @Override public AuthenticationResult createEscalatedAuthenticationResult() { - BasicEscalatorCredential basicEscalatorCredential = getOrUpdateEscalatorCredentials(); - LOG.debug("----------- Creating escalated authentication result. username: %s", basicEscalatorCredential.getUsername()); + LOG.debug("----------- Creating escalated authentication result. username: %s", this.internalClientUsername); // if you found your self asking why the authenticatedBy field is set to null please read this: // https://github.com/apache/incubator-druid/pull/5706#discussion_r185940889 - return new AuthenticationResult( - basicEscalatorCredential.getUsername(), - authorizerName, - null, - null); - } - - public BasicAuthDBConfig getDbConfig() - { - return dbConfig; - } - - private final class BasicEscalatorCredentials implements Credentials - { - @Override - public Request addCredentials(Request builder) - { - BasicEscalatorCredential basicEscalatorCredential = getOrUpdateEscalatorCredentials(); - LOG.debug("----------- Adding escalator credentials. username: %s", basicEscalatorCredential.getUsername()); - return builder.setBasicAuthentication( - basicEscalatorCredential.getUsername(), - basicEscalatorCredential.getPassword() - ); - } - } - - private BasicEscalatorCredential getOrUpdateEscalatorCredentials() - { - BasicEscalatorCredential escalatorCredential = cachedEscalatorCredential.get(); - BasicEscalatorCredential polledEscalatorCredential; - long now = System.currentTimeMillis(); - long cutoff = now - (internalClientCredentialPoll * 1000L); - - if (lastVerified.get() < cutoff) { - lastVerified.set(now); - try { - polledEscalatorCredential = cacheManager.get().getEscalatorCredential(); - } - catch (Exception ex) { - polledEscalatorCredential = null; - } - - if (polledEscalatorCredential == null || polledEscalatorCredential.equals(escalatorCredential)) { - LOG.debug("----------- Escalator credentials validated, no need to modify"); - return escalatorCredential; - } else { - LOG.debug("----------- Modified escalator credentials found, reloading"); - cachedEscalatorCredential.set(polledEscalatorCredential); - return polledEscalatorCredential; - } - } - return escalatorCredential; + return new AuthenticationResult(internalClientUsername, authorizerName, null, null); } } diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheManager.java deleted file mode 100644 index c7fac703b12a..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheManager.java +++ /dev/null @@ -1,43 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.db.cache; - -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; - -/** - * This class is reponsible for maintaining a cache of the escalator database state. The BasicHTTPEscalator - * uses an injected BasicEscalatorCacheManager to set escalator credentials. - */ -public interface BasicEscalatorCacheManager -{ - /** - * Update this cache manager's local state of escalator credential with fresh information pushed by the coordinator. - * - * @param serializedEscalatorCredentialConfig The updated, serialized escalator credential - */ - void handleEscalatorCredentialUpdate(byte[] serializedEscalatorCredentialConfig); - - /** - * Return the cache manager's local view of escalator credential. - * - * @return Escalator credential - */ - BasicEscalatorCredential getEscalatorCredential(); -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheNotifier.java deleted file mode 100644 index 3c07a8877cab..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/BasicEscalatorCacheNotifier.java +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.db.cache; - -/** - * Sends a notification to druid services, containing updated escalator state. - */ -public interface BasicEscalatorCacheNotifier -{ - /** - * Send the escalator credential state contained in updatedEscalatorCredential to all non-coordinator Druid services - * - * @param updatedEscalatorCredential Escalator credential state - */ - void addEscalatorCredentialUpdate(byte[] updatedEscalatorCredential); -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java deleted file mode 100644 index 77b190a1beb3..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorBasicEscalatorCacheNotifier.java +++ /dev/null @@ -1,130 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.db.cache; - -import com.google.common.base.Preconditions; -import com.google.inject.Inject; -import org.apache.druid.concurrent.LifecycleLock; -import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.guice.ManageLifecycle; -import org.apache.druid.guice.annotations.EscalatedClient; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.lifecycle.LifecycleStart; -import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.security.basic.BasicAuthDBConfig; -import org.apache.druid.security.basic.CommonCacheNotifier; -import org.apache.druid.security.basic.authentication.BasicHTTPEscalator; -import org.apache.druid.server.security.Escalator; - -import java.util.concurrent.TimeUnit; - -@ManageLifecycle -public class CoordinatorBasicEscalatorCacheNotifier implements BasicEscalatorCacheNotifier -{ - - private final LifecycleLock lifecycleLock = new LifecycleLock(); - private final CommonCacheNotifier escalatorCredentialCacheNotifier; - - @Inject - public CoordinatorBasicEscalatorCacheNotifier( - DruidNodeDiscoveryProvider discoveryProvider, - final Escalator escalator, - @EscalatedClient HttpClient httpClient - ) - { - escalatorCredentialCacheNotifier = new CommonCacheNotifier( - initEscalatorConfigMap(escalator), - discoveryProvider, - httpClient, - "/druid-ext/basic-security/escalator/listen/credential", - "CoordinatorBasicEscalatorCacheNotifier" - ); - } - - @LifecycleStart - public void start() - { - if (!lifecycleLock.canStart()) { - throw new ISE("can't start."); - } - - try { - escalatorCredentialCacheNotifier.start(); - lifecycleLock.started(); - } - finally { - lifecycleLock.exitStart(); - } - } - - @LifecycleStop - public void stop() - { - if (!lifecycleLock.canStop()) { - return; - } - try { - escalatorCredentialCacheNotifier.stop(); - } - finally { - lifecycleLock.exitStop(); - } - } - - @Override - public void addEscalatorCredentialUpdate(byte[] updatedEscalatorCredential) - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - escalatorCredentialCacheNotifier.addUpdate(updatedEscalatorCredential); - } - - private BasicAuthDBConfig initEscalatorConfigMap(Escalator escalator) - { - Preconditions.checkNotNull(escalator); - - BasicAuthDBConfig dbConfig; - if (escalator instanceof BasicHTTPEscalator) { - BasicHTTPEscalator basicHTTPEscalator = (BasicHTTPEscalator) escalator; - dbConfig = basicHTTPEscalator.getDbConfig(); - } else { - dbConfig = new BasicAuthDBConfig( - null, - null, - null, - null, - null, - true, - BasicAuthDBConfig.DEFAULT_CACHE_NOTIFY_TIMEOUT_MS, - 0, - null, - null, - null, - null, - null, - null, - null, null, - null, - null - ); - } - return dbConfig; - } -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java deleted file mode 100644 index 543afb2b7110..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/CoordinatorPollingBasicEscalatorCacheManager.java +++ /dev/null @@ -1,214 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.db.cache; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.inject.Inject; -import com.google.inject.Injector; -import org.apache.druid.client.coordinator.Coordinator; -import org.apache.druid.concurrent.LifecycleLock; -import org.apache.druid.discovery.DruidLeaderClient; -import org.apache.druid.guice.ManageLifecycle; -import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.RetryUtils; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; -import org.apache.druid.java.util.common.lifecycle.LifecycleStart; -import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.java.util.http.client.Request; -import org.apache.druid.security.basic.BasicAuthCommonCacheConfig; -import org.apache.druid.security.basic.authentication.BasicHTTPEscalator; -import org.apache.druid.security.basic.authentication.BytesFullResponseHandler; -import org.apache.druid.security.basic.authentication.BytesFullResponseHolder; -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; -import org.apache.druid.server.security.Escalator; -import org.jboss.netty.handler.codec.http.HttpMethod; -import org.jboss.netty.handler.codec.http.HttpResponseStatus; -import org.joda.time.Duration; - -import javax.annotation.Nullable; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; - -/** - * Cache manager for non-coordinator services that polls the coordinator for escalator database state. - */ -@ManageLifecycle -public class CoordinatorPollingBasicEscalatorCacheManager implements BasicEscalatorCacheManager -{ - private static final EmittingLogger LOG = new EmittingLogger(CoordinatorPollingBasicEscalatorCacheManager.class); - - private final AtomicReference cachedEscalatorCredential; - private final Injector injector; - private final ObjectMapper objectMapper; - private final LifecycleLock lifecycleLock = new LifecycleLock(); - private final DruidLeaderClient druidLeaderClient; - private final BasicAuthCommonCacheConfig commonCacheConfig; - private final ScheduledExecutorService exec; - - @Inject - public CoordinatorPollingBasicEscalatorCacheManager( - Injector injector, - BasicAuthCommonCacheConfig commonCacheConfig, - @Smile ObjectMapper objectMapper, - @Coordinator DruidLeaderClient druidLeaderClient - ) - { - this.exec = Execs.scheduledSingleThreaded("BasicEscalatorCacheManager-Exec--%d"); - this.injector = injector; - this.commonCacheConfig = commonCacheConfig; - this.objectMapper = objectMapper; - this.druidLeaderClient = druidLeaderClient; - this.cachedEscalatorCredential = new AtomicReference<>(); - } - - @LifecycleStart - public void start() - { - if (!lifecycleLock.canStart()) { - throw new ISE("Can't start."); - } - - LOG.info("Starting CoordinatorPollingBasicEscalatorCacheManager."); - - try { - initEscalatorCredential(); - - ScheduledExecutors.scheduleWithFixedDelay( - exec, - new Duration(commonCacheConfig.getPollingPeriod()), - new Duration(commonCacheConfig.getPollingPeriod()), - () -> { - try { - long randomDelay = ThreadLocalRandom.current().nextLong(0, commonCacheConfig.getMaxRandomDelay()); - LOG.debug("Inserting cachedConfigs random polling delay of [%s] ms", randomDelay); - Thread.sleep(randomDelay); - - LOG.debug("Scheduled escalator credential cache poll is running"); - BasicEscalatorCredential escalatorCredential = fetchEscalatorCredentialFromCoordinator(); - if (escalatorCredential != null) { - cachedEscalatorCredential.set(escalatorCredential); - } - LOG.debug("Scheduled escalator credential cache poll is done"); - } - catch (Throwable t) { - LOG.makeAlert(t, "Error occured while polling for cachedEscalatorCredential.").emit(); - } - } - ); - - lifecycleLock.started(); - LOG.info("Started CoordinatorPollingBasicEscalatorCacheManager."); - } - finally { - lifecycleLock.exitStart(); - } - } - - @LifecycleStop - public void stop() - { - if (!lifecycleLock.canStop()) { - throw new ISE("can't stop."); - } - - LOG.info("CoordinatorPollingBasicEscalatorCacheManager is stopping."); - exec.shutdown(); - LOG.info("CoordinatorPollingBasicEscalatorCacheManager is stopped."); - } - - @Override - public void handleEscalatorCredentialUpdate(byte[] serializedEscalatorCredentialConfig) - { - LOG.debug("Received escalator credential cache update for escalator."); - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - try { - cachedEscalatorCredential.set( - objectMapper.readValue(serializedEscalatorCredentialConfig, BasicEscalatorCredential.class) - ); - } - catch (Exception e) { - LOG.makeAlert(e, "Could not deserialize escalator credential received from coordinator.").emit(); - } - } - - @Override - public BasicEscalatorCredential getEscalatorCredential() - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - - return cachedEscalatorCredential.get(); - } - - @Nullable - private BasicEscalatorCredential fetchEscalatorCredentialFromCoordinator() - { - try { - return RetryUtils.retry( - this::tryFetchEscalatorCredentialFromCoordinator, - e -> true, - commonCacheConfig.getMaxSyncRetries() - ); - } - catch (Exception e) { - LOG.makeAlert(e, "Encountered exception while fetching escalator credential for escalator").emit(); - return null; - } - } - - private BasicEscalatorCredential tryFetchEscalatorCredentialFromCoordinator() throws Exception - { - Request req = druidLeaderClient.makeRequest( - HttpMethod.GET, - "/druid-ext/basic-security/escalator/db/cachedSerializedCredential" - ); - BytesFullResponseHolder responseHolder = (BytesFullResponseHolder) druidLeaderClient.go( - req, - new BytesFullResponseHandler() - ); - if (responseHolder.getStatus().equals(HttpResponseStatus.NOT_FOUND)) { - return null; - } else { - byte[] escalatorCredentialBytes = responseHolder.getBytes(); - return objectMapper.readValue(escalatorCredentialBytes, BasicEscalatorCredential.class); - } - } - - private void initEscalatorCredential() - { - Escalator escalator = injector.getInstance(Escalator.class); - - if (escalator == null) { - return; - } - - if (escalator instanceof BasicHTTPEscalator) { - BasicEscalatorCredential escalatorCredential = fetchEscalatorCredentialFromCoordinator(); - if (escalatorCredential != null) { - cachedEscalatorCredential.set(escalatorCredential); - } - } - } -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/MetadataStoragePollingBasicEscalatorCacheManager.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/MetadataStoragePollingBasicEscalatorCacheManager.java deleted file mode 100644 index b5e656f99e5d..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/cache/MetadataStoragePollingBasicEscalatorCacheManager.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.db.cache; - -import com.google.inject.Inject; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.security.basic.escalator.db.updater.BasicEscalatorMetadataStorageUpdater; -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; - -/** - * Used on coordinator nodes, reading from a BasicEscalatorMetadataStorageUpdater that has direct access to the - * metadata store. - */ -public class MetadataStoragePollingBasicEscalatorCacheManager implements BasicEscalatorCacheManager -{ - private static final Logger log = new Logger(MetadataStoragePollingBasicEscalatorCacheManager.class); - - private final BasicEscalatorMetadataStorageUpdater storageUpdater; - - @Inject - public MetadataStoragePollingBasicEscalatorCacheManager( - BasicEscalatorMetadataStorageUpdater storageUpdater - ) - { - this.storageUpdater = storageUpdater; - - log.info("Starting coordinator basic escalator cache manager."); - } - - @Override - public void handleEscalatorCredentialUpdate(byte[] serializedEscalatorCredentialConfig) - { - } - - @Override - public BasicEscalatorCredential getEscalatorCredential() - { - return storageUpdater.getCachedEscalatorCredential(); - } -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/BasicEscalatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/BasicEscalatorMetadataStorageUpdater.java deleted file mode 100644 index 3289d80763e1..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/BasicEscalatorMetadataStorageUpdater.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.db.updater; - -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; - -/** - * Implementations of this interface are responsible for connecting directly to the metadata storage, - * modifying the escalator database state or reading it. This interface is used by the - * MetadataStoragePollingBasicEscalatorCacheManager (for reads) and the CoordinatorBasicEscalatorResourceHandler - * (for handling configuration read/writes). - */ -public interface BasicEscalatorMetadataStorageUpdater -{ - void updateEscalatorCredential(BasicEscalatorCredential escalatorCredential); - - BasicEscalatorCredential getCachedEscalatorCredential(); - - byte[] getCachedSerializedEscalatorCredential(); - - byte[] getCurrentEscalatorCredentialBytes(); - - void refreshAllNotification(); - -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java deleted file mode 100644 index 5d0d1e0946e5..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/db/updater/CoordinatorBasicEscalatorMetadataStorageUpdater.java +++ /dev/null @@ -1,274 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.db.updater; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.inject.Inject; -import org.apache.druid.common.config.ConfigManager; -import org.apache.druid.concurrent.LifecycleLock; -import org.apache.druid.guice.ManageLifecycle; -import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.concurrent.ScheduledExecutors; -import org.apache.druid.java.util.common.lifecycle.LifecycleStart; -import org.apache.druid.java.util.common.lifecycle.LifecycleStop; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.metadata.MetadataCASUpdate; -import org.apache.druid.metadata.MetadataStorageConnector; -import org.apache.druid.metadata.MetadataStorageTablesConfig; -import org.apache.druid.security.basic.BasicAuthCommonCacheConfig; -import org.apache.druid.security.basic.BasicAuthUtils; -import org.apache.druid.security.basic.escalator.db.cache.BasicEscalatorCacheNotifier; -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredentialBundle; -import org.joda.time.Duration; - -import java.util.Collections; -import java.util.concurrent.Callable; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; - -@ManageLifecycle -public class CoordinatorBasicEscalatorMetadataStorageUpdater implements BasicEscalatorMetadataStorageUpdater -{ - private static final EmittingLogger LOG = new EmittingLogger(CoordinatorBasicEscalatorMetadataStorageUpdater.class); - - - private static final String CREDENTIAL = "credential"; - private static final long UPDATE_RETRY_DELAY = 1000; - - private final MetadataStorageConnector connector; - private final MetadataStorageTablesConfig connectorConfig; - private final BasicAuthCommonCacheConfig commonCacheConfig; - private final ObjectMapper objectMapper; - private final BasicEscalatorCacheNotifier cacheNotifier; - - - private final AtomicReference cachedEscalatorCredential; - private final LifecycleLock lifecycleLock = new LifecycleLock(); - - private final ScheduledExecutorService exec; - private volatile boolean stopped = false; - - @Inject - public CoordinatorBasicEscalatorMetadataStorageUpdater( - MetadataStorageConnector connector, - MetadataStorageTablesConfig connectorConfig, - BasicAuthCommonCacheConfig commonCacheConfig, - @Smile ObjectMapper objectMapper, - BasicEscalatorCacheNotifier cacheNotifier, - ConfigManager configManager // ConfigManager creates the db table we need, set a dependency here - ) - { - this.exec = Execs.scheduledSingleThreaded("CoordinatorBasicEscalatorMetadataStorageUpdater-Exec--%d"); - this.connector = connector; - this.connectorConfig = connectorConfig; - this.commonCacheConfig = commonCacheConfig; - this.objectMapper = objectMapper; - this.cacheNotifier = cacheNotifier; - this.cachedEscalatorCredential = new AtomicReference<>(); - } - - @LifecycleStart - public void start() - { - if (!lifecycleLock.canStart()) { - throw new ISE("can't start."); - } - - try { - LOG.info("Starting CoordinatorBasicEscalatorMetadataStorageUpdater."); - ScheduledExecutors.scheduleWithFixedDelay( - exec, - new Duration(commonCacheConfig.getPollingPeriod()), - new Duration(commonCacheConfig.getPollingPeriod()), - new Callable() - { - @Override - public ScheduledExecutors.Signal call() - { - if (stopped) { - return ScheduledExecutors.Signal.STOP; - } - try { - LOG.debug("Scheduled db escalator credential poll is running"); - byte[] escalatorCredentialBytes = getCurrentEscalatorCredentialBytes(); - - if (escalatorCredentialBytes != null) { - BasicEscalatorCredential escalatorCredential = BasicAuthUtils.deserializeEscalatorCredential( - objectMapper, - escalatorCredentialBytes - ); - if (escalatorCredential != null) { - cachedEscalatorCredential.set(new BasicEscalatorCredentialBundle(escalatorCredential, escalatorCredentialBytes)); - } - } - LOG.debug("Scheduled db escalator credential poll is done"); - } - catch (Throwable t) { - LOG.makeAlert(t, "Error occured while polling for cachedEscalatorCredential.").emit(); - } - return ScheduledExecutors.Signal.REPEAT; - } - } - ); - - lifecycleLock.started(); - } - finally { - lifecycleLock.exitStart(); - } - } - - @LifecycleStop - public void stop() - { - if (!lifecycleLock.canStop()) { - throw new ISE("can't stop."); - } - - LOG.info("CoordinatorBasicEscalatorMetadataStorageUpdater is stopping."); - stopped = true; - LOG.info("CoordinatorBasicEscalatorMetadataStorageUpdater is stopped."); - } - - - - @Override - public void refreshAllNotification() - { - cacheNotifier.addEscalatorCredentialUpdate(cachedEscalatorCredential.get().getSerializedEscalatorCredential()); - } - - @Override - public void updateEscalatorCredential(BasicEscalatorCredential escalatorCredential) - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - - updateEscalatorCredentialInternal(escalatorCredential); - } - - @Override - public BasicEscalatorCredential getCachedEscalatorCredential() - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - - BasicEscalatorCredentialBundle bundle = cachedEscalatorCredential.get(); - if (bundle == null) { - return null; - } else { - return bundle.getEscalatorCredential(); - } - } - - @Override - public byte[] getCachedSerializedEscalatorCredential() - { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - - BasicEscalatorCredentialBundle bundle = cachedEscalatorCredential.get(); - if (bundle == null) { - return null; - } else { - return bundle.getSerializedEscalatorCredential(); - } - } - - @Override - public byte[] getCurrentEscalatorCredentialBytes() - { - return connector.lookup( - connectorConfig.getConfigTable(), - MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, - MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, - getKeyColumn(CREDENTIAL) - ); - } - - private static String getKeyColumn(String keyName) - { - return StringUtils.format("basic_escalator_%s", keyName); - } - - private void updateEscalatorCredentialInternal(BasicEscalatorCredential credential) - { - int numRetries = 5; - int attempts = 0; - while (attempts < numRetries) { - if (updateEscalatorCredentialOnce(credential)) { - return; - } else { - attempts++; - } - try { - Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); - } - catch (InterruptedException ie) { - throw new RuntimeException(ie); - } - } - throw new ISE("Could not update escalator credential due to concurrent update contention."); - } - - private boolean updateEscalatorCredentialOnce(BasicEscalatorCredential credential) - { - byte[] oldValue = getCurrentEscalatorCredentialBytes(); - byte[] newValue = BasicAuthUtils.serializeEscalatorCredential(objectMapper, credential); - return tryUpdateEscalatorCredential(credential, oldValue, newValue); - } - - private boolean tryUpdateEscalatorCredential( - BasicEscalatorCredential credential, - byte[] oldValue, - byte[] newValue - ) - { - try { - MetadataCASUpdate update = new MetadataCASUpdate( - connectorConfig.getConfigTable(), - MetadataStorageConnector.CONFIG_TABLE_KEY_COLUMN, - MetadataStorageConnector.CONFIG_TABLE_VALUE_COLUMN, - getKeyColumn(CREDENTIAL), - oldValue, - newValue - ); - - boolean succeeded = connector.compareAndSwap( - Collections.singletonList(update) - ); - - if (succeeded) { - cachedEscalatorCredential.set(new BasicEscalatorCredentialBundle(credential, newValue)); - cacheNotifier.addEscalatorCredentialUpdate(newValue); - return true; - } else { - return false; - } - } - catch (Exception e) { - throw new RuntimeException(e); - } - } -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResource.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResource.java deleted file mode 100644 index 04e443688889..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResource.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.endpoint; - -import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; -import com.google.inject.Inject; -import com.sun.jersey.spi.container.ResourceFilters; -import org.apache.druid.guice.LazySingleton; -import org.apache.druid.security.basic.BasicSecurityResourceFilter; -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; - -import javax.servlet.http.HttpServletRequest; -import javax.ws.rs.Consumes; -import javax.ws.rs.GET; -import javax.ws.rs.POST; -import javax.ws.rs.Path; -import javax.ws.rs.Produces; -import javax.ws.rs.core.Context; -import javax.ws.rs.core.MediaType; -import javax.ws.rs.core.Response; - -@Path("/druid-ext/basic-security/escalator") -@LazySingleton -public class BasicEscalatorResource -{ - private final BasicEscalatorResourceHandler handler; - - @Inject - public BasicEscalatorResource( - BasicEscalatorResourceHandler handler - ) - { - this.handler = handler; - } - - /** - * @param req HTTP request - * - * @return escalator credential - */ - @GET - @Path("/db/credential") - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response getEscalatorCredential( - @Context HttpServletRequest req - ) - { - return handler.getEscalatorCredential(); - } - - /** - * Update escalator credential - * - * @param req HTTP request - * @param escalatorCredential Escalator credential - * - * @return OK response - */ - @POST - @Path("/db/credential") - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response updateEscalatorCredential( - @Context HttpServletRequest req, - BasicEscalatorCredential escalatorCredential - ) - { - return handler.updateEscalatorCredential(escalatorCredential); - } - - /** - * @param req HTTP request - * - * @return serialized escalator credential - */ - @GET - @Path("/db/cachedSerializedCredential") - @Produces(SmileMediaTypes.APPLICATION_JACKSON_SMILE) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response getCachedSerializedEscalatorCredential( - @Context HttpServletRequest req - ) - { - return handler.getCachedSerializedEscalatorCredential(); - } - - /** - * Listen for escalator credential update notifications for the escalator storage - */ - @POST - @Path("/listen/credential") - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response escalatorCredentialUpdateListener( - @Context HttpServletRequest req, - byte[] serializedEscalatorCredential - ) - { - return handler.escalatorCredentialUpdateListener(serializedEscalatorCredential); - } - - /** - * @param req HTTP request - * - * @return Load status of escalator DB caches - */ - @GET - @Path("/loadStatus") - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response getLoadStatus( - @Context HttpServletRequest req - ) - { - return handler.getLoadStatus(); - } - - /** - * @param req HTTP request - * - * Sends an "update" notification to all services with the current escalator database state, - * causing them to refresh their DB cache state. - */ - @GET - @Path("/refreshAll") - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(BasicSecurityResourceFilter.class) - public Response refreshAll( - @Context HttpServletRequest req - ) - { - return handler.refreshAll(); - } -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResourceHandler.java deleted file mode 100644 index 4dcae6cc91ef..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/BasicEscalatorResourceHandler.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.endpoint; - -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; - -import javax.ws.rs.core.Response; - -/** - * Handles authenticator-related API calls. Coordinator and non-coordinator methods are combined here because of an - * inability to selectively inject jetty resources in configure(Binder binder) of the extension module based - * on node type. - */ -public interface BasicEscalatorResourceHandler -{ - // coordinator methods - Response getEscalatorCredential(); - - Response updateEscalatorCredential(BasicEscalatorCredential escalatorCredential); - - Response getCachedSerializedEscalatorCredential(); - - Response refreshAll(); - - // non-coordinator methods - Response escalatorCredentialUpdateListener(byte[] serializedEscalatorCredential); - - // common methods - Response getLoadStatus(); -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java deleted file mode 100644 index 67f6713dbee3..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/CoordinatorBasicEscalatorResourceHandler.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.endpoint; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; -import com.google.inject.Inject; -import org.apache.druid.guice.annotations.Smile; -import org.apache.druid.security.basic.BasicAuthUtils; -import org.apache.druid.security.basic.BasicSecurityDBResourceException; -import org.apache.druid.security.basic.escalator.db.updater.BasicEscalatorMetadataStorageUpdater; -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; - -import javax.ws.rs.core.Response; - -public class CoordinatorBasicEscalatorResourceHandler implements BasicEscalatorResourceHandler -{ - private final BasicEscalatorMetadataStorageUpdater storageUpdater; - private final ObjectMapper objectMapper; - - @Inject - public CoordinatorBasicEscalatorResourceHandler( - BasicEscalatorMetadataStorageUpdater storageUpdater, - @Smile ObjectMapper objectMapper - ) - { - this.storageUpdater = storageUpdater; - this.objectMapper = objectMapper; - } - - @Override - public Response refreshAll() - { - storageUpdater.refreshAllNotification(); - return Response.ok().build(); - } - - @Override - public Response getEscalatorCredential() - { - BasicEscalatorCredential escalatorCredential = BasicAuthUtils.deserializeEscalatorCredential( - objectMapper, - storageUpdater.getCurrentEscalatorCredentialBytes() - ); - if (escalatorCredential == null) { - return Response.status(Response.Status.NOT_FOUND) - .entity(ImmutableMap.of( - "error", - "Escalator credential does not exist." - )) - .build(); - } else { - BasicEscalatorCredential maskedEscalatorCredential = new BasicEscalatorCredential( - escalatorCredential.getUsername(), - "..." - ); - - return Response.ok(maskedEscalatorCredential).build(); - } - } - - @Override - public Response updateEscalatorCredential(BasicEscalatorCredential escalatorCredential) - { - try { - storageUpdater.updateEscalatorCredential(escalatorCredential); - return Response.ok().build(); - } - catch (BasicSecurityDBResourceException cfe) { - return makeResponseForBasicSecurityDBResourceException(cfe); - } - } - - @Override - public Response getCachedSerializedEscalatorCredential() - { - byte[] cachedSerializedEscalatorCredential = storageUpdater.getCachedSerializedEscalatorCredential(); - if (cachedSerializedEscalatorCredential != null) { - return Response.ok(storageUpdater.getCachedSerializedEscalatorCredential()).build(); - } else { - return Response.status(Response.Status.NOT_FOUND) - .entity(ImmutableMap.of( - "error", - "Escalator credential does not exist." - )) - .build(); - } - } - - @Override - public Response escalatorCredentialUpdateListener(byte[] serializedEscalatorCredential) - { - return Response.status(Response.Status.NOT_FOUND).build(); - } - - @Override - public Response getLoadStatus() - { - Boolean loadStatus = storageUpdater.getCachedEscalatorCredential() != null; - return Response.ok(loadStatus).build(); - } - - private static Response makeResponseForBasicSecurityDBResourceException(BasicSecurityDBResourceException bsre) - { - return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of( - "error", bsre.getMessage() - )) - .build(); - } -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java deleted file mode 100644 index c651b94b5bdd..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/endpoint/DefaultBasicEscalatorResourceHandler.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.endpoint; - -import com.google.inject.Inject; -import org.apache.druid.security.basic.escalator.db.cache.BasicEscalatorCacheManager; -import org.apache.druid.security.basic.escalator.entity.BasicEscalatorCredential; -import org.apache.druid.server.security.AuthenticatorMapper; - -import javax.ws.rs.core.Response; - -public class DefaultBasicEscalatorResourceHandler implements BasicEscalatorResourceHandler -{ - private static final Response NOT_FOUND_RESPONSE = Response.status(Response.Status.NOT_FOUND).build(); - - private final BasicEscalatorCacheManager cacheManager; - - @Inject - public DefaultBasicEscalatorResourceHandler( - BasicEscalatorCacheManager cacheManager, - AuthenticatorMapper authenticatorMapper - ) - { - this.cacheManager = cacheManager; - } - - @Override - public Response refreshAll() - { - return NOT_FOUND_RESPONSE; - } - - @Override - public Response getEscalatorCredential() - { - return NOT_FOUND_RESPONSE; - } - - @Override - public Response updateEscalatorCredential(BasicEscalatorCredential escalatorCredential) - { - return NOT_FOUND_RESPONSE; - } - - @Override - public Response getCachedSerializedEscalatorCredential() - { - return NOT_FOUND_RESPONSE; - } - - @Override - public Response escalatorCredentialUpdateListener(byte[] serializedEscalatorCredential) - { - cacheManager.handleEscalatorCredentialUpdate(serializedEscalatorCredential); - return Response.ok().build(); - } - - @Override - public Response getLoadStatus() - { - Boolean loadStatus = cacheManager.getEscalatorCredential() != null; - return Response.ok(loadStatus).build(); - } -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java deleted file mode 100644 index 6cb68fbb189c..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredential.java +++ /dev/null @@ -1,73 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.entity; - -import com.fasterxml.jackson.annotation.JsonProperty; - -public class BasicEscalatorCredential -{ - private final String username; - private final String password; - - public BasicEscalatorCredential( - @JsonProperty("username") String username, - @JsonProperty("password") String password - ) - { - this.username = username; - this.password = password; - } - - @JsonProperty - public String getUsername() - { - return username; - } - - @JsonProperty - public String getPassword() - { - return password; - } - - @Override - public boolean equals(Object o) - { - if (this == o) { - return true; - } - if (o == null || !getClass().equals(o.getClass())) { - return false; - } - - BasicEscalatorCredential that = (BasicEscalatorCredential) o; - - return (getUsername() != null ? getUsername().equals(that.getUsername()) : that.getUsername() == null) - && (getPassword() != null ? getPassword().equals(that.getPassword()) : that.getPassword() == null); - } - - @Override - public int hashCode() - { - int result = getUsername() != null ? getUsername().hashCode() : 0; - result = 31 * result + (getPassword() != null ? getPassword().hashCode() : 0); - return result; - } -} diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredentialBundle.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredentialBundle.java deleted file mode 100644 index 2108ff0d2b08..000000000000 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/escalator/entity/BasicEscalatorCredentialBundle.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.security.basic.escalator.entity; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -public class BasicEscalatorCredentialBundle -{ - private final BasicEscalatorCredential escalatorCredential; - private final byte[] serializedEscalatorCredential; - - @JsonCreator - public BasicEscalatorCredentialBundle( - @JsonProperty("escalatorCredential") BasicEscalatorCredential credential, - @JsonProperty("serializedEscalatorCredential") byte[] serializedEscalatorCredential - ) - { - this.escalatorCredential = credential; - this.serializedEscalatorCredential = serializedEscalatorCredential; - } - - @JsonProperty - public BasicEscalatorCredential getEscalatorCredential() - { - return escalatorCredential; - } - - @JsonProperty - public byte[] getSerializedEscalatorCredential() - { - return serializedEscalatorCredential; - } -} From 68075d09d13271c0ca2647b1e562596fe531b6db Mon Sep 17 00:00:00 2001 From: Mohammad J Khan Date: Tue, 2 Jul 2019 16:24:27 -0500 Subject: [PATCH 10/24] fixed teamcity inspection deprecated API usage error --- .../rabbitmq/RabbitMQProducerMain.java | 98 +++++++++---------- 1 file changed, 49 insertions(+), 49 deletions(-) diff --git a/extensions-contrib/rabbitmq/src/test/java/org/apache/druid/examples/rabbitmq/RabbitMQProducerMain.java b/extensions-contrib/rabbitmq/src/test/java/org/apache/druid/examples/rabbitmq/RabbitMQProducerMain.java index 78c4c220995b..3fdc550bbbe8 100644 --- a/extensions-contrib/rabbitmq/src/test/java/org/apache/druid/examples/rabbitmq/RabbitMQProducerMain.java +++ b/extensions-contrib/rabbitmq/src/test/java/org/apache/druid/examples/rabbitmq/RabbitMQProducerMain.java @@ -52,67 +52,67 @@ public static void main(String[] args) // We use a List to keep track of option insertion order. See below. final List