You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "paul-rogers (via GitHub)" <gi...@apache.org> on 2023/03/31 20:50:31 UTC

[GitHub] [druid] paul-rogers opened a new pull request, #14009: Add basic security functions to druidapi

paul-rogers opened a new pull request, #14009:
URL: https://github.com/apache/druid/pull/14009

   This PR adds a complete set of Basic security functions to the Python `druidapi`. These functions are handy for setting up security, inspecting the security setup, and learning the nuances of the basic security system. They would make a fine foundation for Basic security tutorial notebook. If we did such a notebook:
   
   * Emphasize that users are defined twice: once in the authorizer, again in the authenticator.
   * The many config settings that have to be done just right.
   * The complexities of SQL security: sometimes one needs multiple permissions.
   
   Since the Druid console doesn't provide tools to set up basic security, doing it via Python is a handy way to get started until a user defines a more production-grade integration with an external system.
   
   #### Example:
   
   ```python
   # Define a coordinator-specific client, using the admin user
   coord = druidapi.jupyter_client('http://localhost:8081', auth=('admin', 'pwd'))
   # Create a basic auth client for your authenticator and authorizer:
   ac = coord.basic_security('myAuthorizer', 'myAuthenticator')
   
   # Get information
   # List users
   ac.users()
   # List roles
   ac.users()
   # List roles for a user
   ac.authorization_user('alice')
   # List permissions for a role
   ac.role_permissions('aliceRole')
   
   # Create user
   ac.add_user('fred', 'pwd')
   # Create role
   ac.add_role('myRole')
   # Grant permissions to a role
   perms = [ac.resource_action(consts.DATASOURCE_RESOURCE, 'foo', consts.READ_ACTION)]
   ac.set_role_permissions('myRole', perms)
   # Assign a role to a user
   ac.assign_role_to_user('myRole', 'fred')
   
   # "Log in" as the new user
   fred = druidapi.jupyter_client('http://localhost:8888', auth=('fred', 'pwd'))
   # Perform operations as the user.
   fred.sql.sql('SELECT * FROM foo LIMIT 10')
   
   # Drop user
   ac.drop_user('fred')
   ```
   
   #### Release note
   
   See the description.
   
   <hr>
   
   This PR has:
   
   - [X] been self-reviewed.
   - [X] added documentation for new or modified features or behaviors.
   - [X] a release note entry in the PR description.
   - [X] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [ ] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [X] been tested in a test Druid cluster.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] paul-rogers commented on pull request #14009: Add basic security functions to druidapi

Posted by "paul-rogers (via GitHub)" <gi...@apache.org>.
paul-rogers commented on PR #14009:
URL: https://github.com/apache/druid/pull/14009#issuecomment-1496491953

   @vtlim, thanks again for acting as my spell-checker and proof-reader! 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] vtlim commented on a diff in pull request #14009: Add basic security functions to druidapi

Posted by "vtlim (via GitHub)" <gi...@apache.org>.
vtlim commented on code in PR #14009:
URL: https://github.com/apache/druid/pull/14009#discussion_r1156462105


##########
docs/operations/security-overview.md:
##########
@@ -176,70 +170,82 @@ An example configuration:
    druid.auth.authorizer.MyBasicMetadataAuthorizer.type=basic
    ```
 
-3. Restart the cluster. 
+3. Restart the cluster.
 
-See [Authentication and Authorization](../design/auth.md) for more information about the Authenticator, Escalator, and Authorizer concepts. See [Basic Security](../development/extensions-core/druid-basic-security.md) for more information about the extension used in the examples above, and [Kerberos](../development/extensions-core/druid-kerberos.md) for Kerberos authentication.
+See

Review Comment:
   ```suggestion
   See the following topics for more information:
   ```



##########
docs/operations/security-overview.md:
##########
@@ -176,70 +170,82 @@ An example configuration:
    druid.auth.authorizer.MyBasicMetadataAuthorizer.type=basic
    ```
 
-3. Restart the cluster. 
+3. Restart the cluster.
 
-See [Authentication and Authorization](../design/auth.md) for more information about the Authenticator, Escalator, and Authorizer concepts. See [Basic Security](../development/extensions-core/druid-basic-security.md) for more information about the extension used in the examples above, and [Kerberos](../development/extensions-core/druid-kerberos.md) for Kerberos authentication.
+See
 
+* [Authentication and Authorization](../design/auth.md) for more information about the Authenticator,
+Escalator, and Authorizer concepts.
+* [Basic Security](../development/extensions-core/druid-basic-security.md) for more information about
+the extension used in the examples above.
+* [Kerberos](../development/extensions-core/druid-kerberos.md) for Kerberos authentication.
+* [User authentication and authorization](security-user-auth.md) for details of permissions.
+* [SQL permissions](security-user-auth.md#sql-permissions) for permissions on SQL system tables.
+* The `druidapi` Python library, provided as part of the Druid tutorisls, for functions you can

Review Comment:
   ```suggestion
   * [The `druidapi` Python library](../tutorials/tutorial-jupyter-index.md#python-api-for-druid), provided as part of the Druid tutorials, for functions you can
   ```



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/druid.py:
##########
@@ -116,6 +117,33 @@ def datasources(self) -> DatasourceClient:
         if not self.datasource_client:
             self.datasource_client = DatasourceClient(self.rest_client)
         return self.datasource_client
+    
+    def basic_security(self, authenticator, authorizer=None):
+        '''
+        Returns a client to work with a basic authorization authenticator/authorizer pair.
+        This client assumes the typical case of one authenticator and one authorizer. If
+        you have more than one, create multiple clients.
+
+        The basic security API is not proxied through the router: it must work directly with
+        the coordinator. Create an ad-hoc Druid client for your coordinator. Because you have
+        Basic security enabled, you must specify the admin user and password:
+
+        ```
+        coord = druidapi.jupyter_client('http://localhost:8081', auth=('admin', 'admin-pwd'))
+        ac = coord.basic_security('yourAuthenticator', 'yourAuthorizer')
+        ```
+
+        Parameters
+        ----------
+        authenticator: str
+            Authenticator name as set in the `druid.auth.authenticatorChain`
+            runtime property.
+
+        authorizer: str, default = authenticator
+            Authorizor name as set in the `druid.auth.authorizers` runtime property.
+            Defaults to the same as `authenticator` for simple cases.

Review Comment:
   Is the default value of `authorizer` the string "authenticator", or is the default value the same value of what's assigned to `authenticator`?



##########
docs/operations/security-overview.md:
##########
@@ -176,70 +170,82 @@ An example configuration:
    druid.auth.authorizer.MyBasicMetadataAuthorizer.type=basic
    ```
 
-3. Restart the cluster. 
+3. Restart the cluster.
 
-See [Authentication and Authorization](../design/auth.md) for more information about the Authenticator, Escalator, and Authorizer concepts. See [Basic Security](../development/extensions-core/druid-basic-security.md) for more information about the extension used in the examples above, and [Kerberos](../development/extensions-core/druid-kerberos.md) for Kerberos authentication.
+See
 
+* [Authentication and Authorization](../design/auth.md) for more information about the Authenticator,
+Escalator, and Authorizer concepts.

Review Comment:
   ```suggestion
   Escalator, and Authorizer.
   ```



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/druid.py:
##########
@@ -116,6 +117,33 @@ def datasources(self) -> DatasourceClient:
         if not self.datasource_client:
             self.datasource_client = DatasourceClient(self.rest_client)
         return self.datasource_client
+    
+    def basic_security(self, authenticator, authorizer=None):
+        '''
+        Returns a client to work with a basic authorization authenticator/authorizer pair.
+        This client assumes the typical case of one authenticator and one authorizer. If
+        you have more than one, create multiple clients.
+
+        The basic security API is not proxied through the router: it must work directly with
+        the coordinator. Create an ad-hoc Druid client for your coordinator. Because you have
+        Basic security enabled, you must specify the admin user and password:
+
+        ```
+        coord = druidapi.jupyter_client('http://localhost:8081', auth=('admin', 'admin-pwd'))
+        ac = coord.basic_security('yourAuthenticator', 'yourAuthorizer')
+        ```
+
+        Parameters
+        ----------
+        authenticator: str
+            Authenticator name as set in the `druid.auth.authenticatorChain`
+            runtime property.
+
+        authorizer: str, default = authenticator
+            Authorizor name as set in the `druid.auth.authorizers` runtime property.

Review Comment:
   ```suggestion
               Authorizer name as set in the `druid.auth.authorizers` runtime property.
   ```



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/basic_auth.py:
##########
@@ -0,0 +1,238 @@
+# 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.
+
+BASIC_AUTH_BASE = '/druid-ext/basic-security'
+
+AUTHENTICATION_BASE = BASIC_AUTH_BASE + '/authentication'
+REQ_AUTHENTICATION_LOAD_STATUS = AUTHENTICATION_BASE + '/loadStatus'
+REQ_AUTHENTICATION_REFRESH_ALL = AUTHENTICATION_BASE + '/refreshAll'
+AUTHENTICATOR_BASE = AUTHENTICATION_BASE + '/db/{}'
+REQ_AUTHENTICATION_USERS = AUTHENTICATOR_BASE + '/users'
+REQ_AUTHENTICATION_USER = REQ_AUTHENTICATION_USERS + '/{}'
+REQ_AUTHENTICATION_CREDENTIALS = REQ_AUTHENTICATION_USER + '/credentials'
+
+AUTHORIZATION_BASE = BASIC_AUTH_BASE + '/authorization'
+REQ_AUTHORIZATION_LOAD_STATUS = AUTHORIZATION_BASE + '/loadStatus'
+REQ_AUTHORIZATION_REFRESH_ALL = AUTHORIZATION_BASE + '/refreshAll'
+AUTHORIZATION_BASE = AUTHORIZATION_BASE + '/db/{}'
+REQ_AUTHORIZATION_USERS = AUTHORIZATION_BASE + '/users'
+REQ_AUTHORIZATION_USER = REQ_AUTHORIZATION_USERS + '/{}'
+REQ_AUTHORIZATION_USER_ROLES = REQ_AUTHORIZATION_USER + '/roles'
+REQ_AUTHORIZATION_USER_ROLE = REQ_AUTHORIZATION_USER_ROLES + '/{}'
+REQ_AUTHORIZATION_GROUP_MAPPINGS = AUTHORIZATION_BASE + '/groupMappings'
+REQ_AUTHORIZATION_GROUP_MAPPING = AUTHORIZATION_BASE + '/groupMappings/{}'
+REQ_AUTHORIZATION_GROUP_ROLES = REQ_AUTHORIZATION_GROUP_MAPPING + '/roles'
+REQ_AUTHORIZATION_GROUP_ROLE = REQ_AUTHORIZATION_GROUP_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLES = AUTHORIZATION_BASE + '/roles'
+REQ_AUTHORIZATION_ROLE = REQ_AUTHORIZATION_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLE_PERMISSIONS = REQ_AUTHORIZATION_ROLE + '/permissions'
+REQ_USER_MAP = AUTHORIZATION_BASE + '/cachedSerializedUserMap'
+
+class BasicAuthClient:
+    '''
+    Manage Basic security. The Druid session must be logged in with the super
+    user, or some other user who has permission to modify user credentials.
+
+    Each client works with one authorizor/authenticator pair. Create multiple clients if you have to

Review Comment:
   ```suggestion
       Each client works with one authorizer/authenticator pair. Create multiple clients to
   ```



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/druid.py:
##########
@@ -116,6 +117,33 @@ def datasources(self) -> DatasourceClient:
         if not self.datasource_client:
             self.datasource_client = DatasourceClient(self.rest_client)
         return self.datasource_client
+    
+    def basic_security(self, authenticator, authorizer=None):
+        '''
+        Returns a client to work with a basic authorization authenticator/authorizer pair.
+        This client assumes the typical case of one authenticator and one authorizer. If
+        you have more than one, create multiple clients.
+
+        The basic security API is not proxied through the router: it must work directly with
+        the coordinator. Create an ad-hoc Druid client for your coordinator. Because you have

Review Comment:
   ```suggestion
           The basic security API is not proxied through the Router: it must work directly with
           the Coordinator. Create an ad hoc Druid client for your Coordinator. Because you have
   ```



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/rest.py:
##########
@@ -95,10 +95,27 @@ class DruidRestClient:
     concatenating the service endpoint with the request URL.
     '''
 
-    def __init__(self, endpoint):
+    def __init__(self, endpoint, auth=None):
+        '''
+        Creates a Druid rest client endpoint using the given endpoint URI and
+        optional authentication.
+
+        Parameters
+        ----------
+        endpoint: str
+            The Druid router endpoint of the form 'server:port'. Use
+            'localhost:8888' for a Druid running locally.
+
+        auth: str, defult = None
+            Optional authorization credentials in the format describd
+            by the Requets library. For Basic auth use

Review Comment:
   ```suggestion
               by the Requests library. For Basic auth use
   ```



##########
docs/operations/security-overview.md:
##########
@@ -176,70 +170,82 @@ An example configuration:
    druid.auth.authorizer.MyBasicMetadataAuthorizer.type=basic
    ```
 
-3. Restart the cluster. 
+3. Restart the cluster.
 
-See [Authentication and Authorization](../design/auth.md) for more information about the Authenticator, Escalator, and Authorizer concepts. See [Basic Security](../development/extensions-core/druid-basic-security.md) for more information about the extension used in the examples above, and [Kerberos](../development/extensions-core/druid-kerberos.md) for Kerberos authentication.
+See
 
+* [Authentication and Authorization](../design/auth.md) for more information about the Authenticator,
+Escalator, and Authorizer concepts.
+* [Basic Security](../development/extensions-core/druid-basic-security.md) for more information about
+the extension used in the examples above.
+* [Kerberos](../development/extensions-core/druid-kerberos.md) for Kerberos authentication.
+* [User authentication and authorization](security-user-auth.md) for details of permissions.

Review Comment:
   ```suggestion
   * [User authentication and authorization](security-user-auth.md) for details about permissions.
   ```



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/basic_auth.py:
##########
@@ -0,0 +1,238 @@
+# 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.
+
+BASIC_AUTH_BASE = '/druid-ext/basic-security'
+
+AUTHENTICATION_BASE = BASIC_AUTH_BASE + '/authentication'
+REQ_AUTHENTICATION_LOAD_STATUS = AUTHENTICATION_BASE + '/loadStatus'
+REQ_AUTHENTICATION_REFRESH_ALL = AUTHENTICATION_BASE + '/refreshAll'
+AUTHENTICATOR_BASE = AUTHENTICATION_BASE + '/db/{}'
+REQ_AUTHENTICATION_USERS = AUTHENTICATOR_BASE + '/users'
+REQ_AUTHENTICATION_USER = REQ_AUTHENTICATION_USERS + '/{}'
+REQ_AUTHENTICATION_CREDENTIALS = REQ_AUTHENTICATION_USER + '/credentials'
+
+AUTHORIZATION_BASE = BASIC_AUTH_BASE + '/authorization'
+REQ_AUTHORIZATION_LOAD_STATUS = AUTHORIZATION_BASE + '/loadStatus'
+REQ_AUTHORIZATION_REFRESH_ALL = AUTHORIZATION_BASE + '/refreshAll'
+AUTHORIZATION_BASE = AUTHORIZATION_BASE + '/db/{}'
+REQ_AUTHORIZATION_USERS = AUTHORIZATION_BASE + '/users'
+REQ_AUTHORIZATION_USER = REQ_AUTHORIZATION_USERS + '/{}'
+REQ_AUTHORIZATION_USER_ROLES = REQ_AUTHORIZATION_USER + '/roles'
+REQ_AUTHORIZATION_USER_ROLE = REQ_AUTHORIZATION_USER_ROLES + '/{}'
+REQ_AUTHORIZATION_GROUP_MAPPINGS = AUTHORIZATION_BASE + '/groupMappings'
+REQ_AUTHORIZATION_GROUP_MAPPING = AUTHORIZATION_BASE + '/groupMappings/{}'
+REQ_AUTHORIZATION_GROUP_ROLES = REQ_AUTHORIZATION_GROUP_MAPPING + '/roles'
+REQ_AUTHORIZATION_GROUP_ROLE = REQ_AUTHORIZATION_GROUP_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLES = AUTHORIZATION_BASE + '/roles'
+REQ_AUTHORIZATION_ROLE = REQ_AUTHORIZATION_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLE_PERMISSIONS = REQ_AUTHORIZATION_ROLE + '/permissions'
+REQ_USER_MAP = AUTHORIZATION_BASE + '/cachedSerializedUserMap'
+
+class BasicAuthClient:
+    '''
+    Manage Basic security. The Druid session must be logged in with the super
+    user, or some other user who has permission to modify user credentials.
+
+    Each client works with one authorizor/authenticator pair. Create multiple clients if you have to
+    work with multiple authenticators on a single server.
+
+    The basic pattern to add users and permissions is:
+
+    ```
+    # Create a client for your coordinator (Basic auth is not proxied through the router)
+    coord = druidapi.jupyter_client('http://localhost:8081', auth=('admin', 'password'))
+
+    # Get a client for your authenticator and authorizer:
+    ac = coord.basic_security('yourAuthorizer', 'yourAuthenticator')
+
+    # Create a user in both the authenticator and authorizer
+    ac.add_user('bob', 'secret')
+
+    # Define a role
+    ac.add_role('myRole')
+
+    # Assign the role to the user
+    ac.assign_role_to_user('myRole', 'bob')
+
+    # Give the role some permissions
+    ac.grant_permissions('myRole', [[consts.DATASOURCE_RESOURCE, 'foo', consts.READ_ACTION]])
+    ```
+
+    Then use the various other methods to list users, roles and permissions to verify the
+    setup. You can now create a second Druid client that acts as the new user:

Review Comment:
   ```suggestion
       setup. You can then create a second Druid client that acts as the new user:
   ```



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/basic_auth.py:
##########
@@ -0,0 +1,238 @@
+# 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.
+
+BASIC_AUTH_BASE = '/druid-ext/basic-security'
+
+AUTHENTICATION_BASE = BASIC_AUTH_BASE + '/authentication'
+REQ_AUTHENTICATION_LOAD_STATUS = AUTHENTICATION_BASE + '/loadStatus'
+REQ_AUTHENTICATION_REFRESH_ALL = AUTHENTICATION_BASE + '/refreshAll'
+AUTHENTICATOR_BASE = AUTHENTICATION_BASE + '/db/{}'
+REQ_AUTHENTICATION_USERS = AUTHENTICATOR_BASE + '/users'
+REQ_AUTHENTICATION_USER = REQ_AUTHENTICATION_USERS + '/{}'
+REQ_AUTHENTICATION_CREDENTIALS = REQ_AUTHENTICATION_USER + '/credentials'
+
+AUTHORIZATION_BASE = BASIC_AUTH_BASE + '/authorization'
+REQ_AUTHORIZATION_LOAD_STATUS = AUTHORIZATION_BASE + '/loadStatus'
+REQ_AUTHORIZATION_REFRESH_ALL = AUTHORIZATION_BASE + '/refreshAll'
+AUTHORIZATION_BASE = AUTHORIZATION_BASE + '/db/{}'
+REQ_AUTHORIZATION_USERS = AUTHORIZATION_BASE + '/users'
+REQ_AUTHORIZATION_USER = REQ_AUTHORIZATION_USERS + '/{}'
+REQ_AUTHORIZATION_USER_ROLES = REQ_AUTHORIZATION_USER + '/roles'
+REQ_AUTHORIZATION_USER_ROLE = REQ_AUTHORIZATION_USER_ROLES + '/{}'
+REQ_AUTHORIZATION_GROUP_MAPPINGS = AUTHORIZATION_BASE + '/groupMappings'
+REQ_AUTHORIZATION_GROUP_MAPPING = AUTHORIZATION_BASE + '/groupMappings/{}'
+REQ_AUTHORIZATION_GROUP_ROLES = REQ_AUTHORIZATION_GROUP_MAPPING + '/roles'
+REQ_AUTHORIZATION_GROUP_ROLE = REQ_AUTHORIZATION_GROUP_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLES = AUTHORIZATION_BASE + '/roles'
+REQ_AUTHORIZATION_ROLE = REQ_AUTHORIZATION_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLE_PERMISSIONS = REQ_AUTHORIZATION_ROLE + '/permissions'
+REQ_USER_MAP = AUTHORIZATION_BASE + '/cachedSerializedUserMap'
+
+class BasicAuthClient:
+    '''
+    Manage Basic security. The Druid session must be logged in with the super
+    user, or some other user who has permission to modify user credentials.
+
+    Each client works with one authorizor/authenticator pair. Create multiple clients if you have to
+    work with multiple authenticators on a single server.
+
+    The basic pattern to add users and permissions is:
+
+    ```
+    # Create a client for your coordinator (Basic auth is not proxied through the router)
+    coord = druidapi.jupyter_client('http://localhost:8081', auth=('admin', 'password'))
+
+    # Get a client for your authenticator and authorizer:
+    ac = coord.basic_security('yourAuthorizer', 'yourAuthenticator')
+
+    # Create a user in both the authenticator and authorizer
+    ac.add_user('bob', 'secret')
+
+    # Define a role
+    ac.add_role('myRole')
+
+    # Assign the role to the user
+    ac.assign_role_to_user('myRole', 'bob')
+
+    # Give the role some permissions
+    ac.grant_permissions('myRole', [[consts.DATASOURCE_RESOURCE, 'foo', consts.READ_ACTION]])
+    ```
+
+    Then use the various other methods to list users, roles and permissions to verify the
+    setup. You can now create a second Druid client that acts as the new user:
+
+    ```
+    bob_client = druidapi.jupyter_client('http://localhost:8888', auth=('bob', 'secret'))
+    ```
+
+    See https://druid.apache.org/docs/latest/operations/security-overview.html#enable-authorizers
+    '''
+
+    def __init__(self, rest_client, authenticator, authorizer=None):
+        self.rest_client = rest_client
+        self.authenticator = authenticator
+        self.authorizer = authorizer if authorizer else authenticator
+
+    # Authentication
+
+    def authentication_status(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_LOAD_STATUS)
+
+    def authentication_refresh(self) -> None:
+        self.rest_client.get(REQ_AUTHENTICATION_REFRESH_ALL)
+
+    def create_authentication_user(self, user) -> None:
+        self.rest_client.post(REQ_AUTHENTICATION_USER, None, args=[self.authenticator, user])
+
+    def set_password(self, user, password) -> None:
+        self.rest_client.post_only_json(REQ_AUTHENTICATION_CREDENTIALS, {'password': password}, args=[self.authenticator, user])
+
+    def drop_authentication_user(self, user) -> None:
+        self.rest_client.delete(REQ_AUTHENTICATION_USER, args=[self.authenticator, user])
+    
+    def authentication_user(self, user) -> dict:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_USER, args=[self.authenticator, user])
+    
+    def authentication_users(self) -> list:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_USERS, args=[self.authenticator])
+    
+    # Authorization
+    # Groups are not documented. Use at your own risk.
+
+    def authorization_status(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_LOAD_STATUS)
+
+    def authorization_refresh(self) -> None:
+        self.rest_client.get(REQ_AUTHORIZATION_REFRESH_ALL)
+
+    def create_authorization_user(self, user) -> None:
+        self.rest_client.post(REQ_AUTHORIZATION_USER, None, args=[self.authorizer, user])
+
+    def drop_authorization_user(self, user) -> None:
+        self.rest_client.delete(REQ_AUTHORIZATION_USER, args=[self.authenticator, user])
+    
+    def authorization_user(self, user) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_USER, args=[self.authorizer, user])
+    
+    def authorization_users(self) -> list:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_USERS, args=[self.authorizer])
+    
+    def create_group(self, group, payload):
+        self.rest_client.post_json(REQ_AUTHORIZATION_GROUP_MAPPING, payload, args=[self.authorizer, group])
+
+    def drop_group(self, group):
+        self.rest_client.delete(REQ_AUTHORIZATION_GROUP_MAPPING, args=[self.authorizer, group])
+
+    def groups(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_GROUP_MAPPINGS, args=[self.authorizer])
+
+    def group(self, group) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_GROUP_MAPPING, args=[self.authorizer, group])
+    
+    def roles(self):
+        return self.rest_client.get_json(REQ_AUTHORIZATION_ROLES, args=[self.authenticator])
+ 
+    def add_role(self, role):
+        self.rest_client.post(REQ_AUTHORIZATION_ROLE, None, args=[self.authenticator, role])
+   
+    def drop_role(self, role):
+        self.rest_client.delete(REQ_AUTHORIZATION_ROLE, args=[self.authorizer, role])
+
+    def set_role_permissions(self, role, permissions):
+        self.rest_client.post_only_json(REQ_AUTHORIZATION_ROLE_PERMISSIONS, permissions, args=[self.authenticator, role])
+
+    def role_permissions(self, role):
+        return self.rest_client.get_json(REQ_AUTHORIZATION_ROLE_PERMISSIONS, args=[self.authenticator, role])
+    
+    def assign_role_to_user(self, role, user):
+        self.rest_client.post(REQ_AUTHORIZATION_USER_ROLE, None, args=[self.authenticator, user, role])
+
+    def revoke_role_from_user(self, role, user):
+        self.rest_client.delete(REQ_AUTHORIZATION_USER_ROLE, args=[self.authenticator, user, role])
+
+    def assign_role_to_group(self, group, role):
+        self.rest_client.post(REQ_AUTHORIZATION_GROUP_ROLE, None, args=[self.authenticator, group, role])
+
+    def revoke_role_from_group(self, group, role):
+        self.rest_client.delete(REQ_AUTHORIZATION_GROUP_ROLE, args=[self.authenticator, group, role])
+
+    def user_map(self):
+        # Result uses Smile encoding, not JSON. This is really just for sanity
+        # checks: a Python client can't make use of the info.
+        # To decode, see newsmile: https://pypi.org/project/newsmile/
+        # However, the format Druid returns is not quite compatible with newsmile
+        return self.rest_client.get(REQ_USER_MAP, args=[self.authenticator])
+
+    # Convenience methods
+
+    def add_user(self, user, password):
+        '''
+        Adds a user to both the authenticator and authorizer.
+        '''
+        self.create_authentication_user(user)
+        self.set_password(user, password)
+        self.create_authorization_user(user)
+
+    def drop_user(self, user):
+        '''
+        Drops a user from both the authenticator and authorizer.
+        '''
+        self.drop_authorization_user(user)
+        self.drop_authentication_user(user)
+
+    def users(self):
+        '''
+        Returns the list of authenticator and authorizer users.
+        '''
+        return {
+            "authenticator": self.authorization_users(),
+            "authorizer": self.authentication_users()
+        }
+
+    def status(self):
+        '''
+        Returns both the authenticator and authorizer status.
+        '''
+        return {
+            "authenticator": self.authorization_status(),
+            "authorizer": self.authentication_status()
+        }
+    
+    def resource(self, type, name):
+        return {
+            'type': type,
+            'name': name
+        }
+    
+    def action(self, resource, action):
+        return {
+            'resource': resource,
+            'action': action
+        }
+
+    def resource_action(self, type, name, action):
+        return self.action(self.resource(type, name), action)
+
+    def grant_permissions(self, role, triples):
+        '''
+        Set the permissions for a role given an array of triples of the form
+        [type, name, action].

Review Comment:
   May be helpful to add an example here. Is it an array or arrays or an array of tuples (or either)? Based on the loop, maybe something like this?
   ```
   [[type0, name0, action0], [type1, name1, action1]]
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekrb19 commented on a diff in pull request #14009: Add basic security functions to druidapi

Posted by "abhishekrb19 (via GitHub)" <gi...@apache.org>.
abhishekrb19 commented on code in PR #14009:
URL: https://github.com/apache/druid/pull/14009#discussion_r1160816365


##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/basic_auth.py:
##########
@@ -0,0 +1,238 @@
+# 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.
+
+BASIC_AUTH_BASE = '/druid-ext/basic-security'
+
+AUTHENTICATION_BASE = BASIC_AUTH_BASE + '/authentication'
+REQ_AUTHENTICATION_LOAD_STATUS = AUTHENTICATION_BASE + '/loadStatus'
+REQ_AUTHENTICATION_REFRESH_ALL = AUTHENTICATION_BASE + '/refreshAll'
+AUTHENTICATOR_BASE = AUTHENTICATION_BASE + '/db/{}'
+REQ_AUTHENTICATION_USERS = AUTHENTICATOR_BASE + '/users'
+REQ_AUTHENTICATION_USER = REQ_AUTHENTICATION_USERS + '/{}'
+REQ_AUTHENTICATION_CREDENTIALS = REQ_AUTHENTICATION_USER + '/credentials'
+
+AUTHORIZATION_BASE = BASIC_AUTH_BASE + '/authorization'
+REQ_AUTHORIZATION_LOAD_STATUS = AUTHORIZATION_BASE + '/loadStatus'
+REQ_AUTHORIZATION_REFRESH_ALL = AUTHORIZATION_BASE + '/refreshAll'
+AUTHORIZATION_BASE = AUTHORIZATION_BASE + '/db/{}'
+REQ_AUTHORIZATION_USERS = AUTHORIZATION_BASE + '/users'
+REQ_AUTHORIZATION_USER = REQ_AUTHORIZATION_USERS + '/{}'
+REQ_AUTHORIZATION_USER_ROLES = REQ_AUTHORIZATION_USER + '/roles'
+REQ_AUTHORIZATION_USER_ROLE = REQ_AUTHORIZATION_USER_ROLES + '/{}'
+REQ_AUTHORIZATION_GROUP_MAPPINGS = AUTHORIZATION_BASE + '/groupMappings'
+REQ_AUTHORIZATION_GROUP_MAPPING = AUTHORIZATION_BASE + '/groupMappings/{}'
+REQ_AUTHORIZATION_GROUP_ROLES = REQ_AUTHORIZATION_GROUP_MAPPING + '/roles'
+REQ_AUTHORIZATION_GROUP_ROLE = REQ_AUTHORIZATION_GROUP_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLES = AUTHORIZATION_BASE + '/roles'
+REQ_AUTHORIZATION_ROLE = REQ_AUTHORIZATION_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLE_PERMISSIONS = REQ_AUTHORIZATION_ROLE + '/permissions'
+REQ_USER_MAP = AUTHORIZATION_BASE + '/cachedSerializedUserMap'
+
+class BasicAuthClient:
+    '''
+    Manage Basic security. The Druid session must be logged in with the super
+    user, or some other user who has permission to modify user credentials.
+
+    Each client works with one authorizer/authenticator pair. Create multiple clients if you have to
+    work with multiple authenticators on a single server.
+
+    The basic pattern to add users and permissions is:
+
+    ```
+    # Create a client for your coordinator (Basic auth is not proxied through the router)
+    coord = druidapi.jupyter_client('http://localhost:8081', auth=('admin', 'password'))
+
+    # Get a client for your authenticator and authorizer:
+    ac = coord.basic_security('yourAuthorizer', 'yourAuthenticator')
+
+    # Create a user in both the authenticator and authorizer
+    ac.add_user('bob', 'secret')
+
+    # Define a role
+    ac.add_role('myRole')
+
+    # Assign the role to the user
+    ac.assign_role_to_user('myRole', 'bob')
+
+    # Give the role some permissions
+    ac.grant_permissions('myRole', [[consts.DATASOURCE_RESOURCE, 'foo', consts.READ_ACTION]])
+    ```
+
+    Then use the various other methods to list users, roles and permissions to verify the
+    setup. You can then create a second Druid client that acts as the new user:
+
+    ```
+    bob_client = druidapi.jupyter_client('http://localhost:8888', auth=('bob', 'secret'))
+    ```
+
+    See https://druid.apache.org/docs/latest/operations/security-overview.html#enable-authorizers
+    '''
+
+    def __init__(self, rest_client, authenticator, authorizer=None):
+        self.rest_client = rest_client
+        self.authenticator = authenticator
+        self.authorizer = authorizer if authorizer else authenticator
+
+    # Authentication
+
+    def authentication_status(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_LOAD_STATUS)
+
+    def authentication_refresh(self) -> None:
+        self.rest_client.get(REQ_AUTHENTICATION_REFRESH_ALL)
+
+    def create_authentication_user(self, user) -> None:
+        self.rest_client.post(REQ_AUTHENTICATION_USER, None, args=[self.authenticator, user])
+
+    def set_password(self, user, password) -> None:
+        self.rest_client.post_only_json(REQ_AUTHENTICATION_CREDENTIALS, {'password': password}, args=[self.authenticator, user])
+
+    def drop_authentication_user(self, user) -> None:
+        self.rest_client.delete(REQ_AUTHENTICATION_USER, args=[self.authenticator, user])
+    
+    def authentication_user(self, user) -> dict:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_USER, args=[self.authenticator, user])
+    
+    def authentication_users(self) -> list:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_USERS, args=[self.authenticator])
+    
+    # Authorization
+    # Groups are not documented. Use at your own risk.
+
+    def authorization_status(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_LOAD_STATUS)
+
+    def authorization_refresh(self) -> None:
+        self.rest_client.get(REQ_AUTHORIZATION_REFRESH_ALL)
+
+    def create_authorization_user(self, user) -> None:
+        self.rest_client.post(REQ_AUTHORIZATION_USER, None, args=[self.authorizer, user])
+
+    def drop_authorization_user(self, user) -> None:
+        self.rest_client.delete(REQ_AUTHORIZATION_USER, args=[self.authenticator, user])
+    
+    def authorization_user(self, user) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_USER, args=[self.authorizer, user])
+    
+    def authorization_users(self) -> list:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_USERS, args=[self.authorizer])
+    
+    def create_group(self, group, payload):
+        self.rest_client.post_json(REQ_AUTHORIZATION_GROUP_MAPPING, payload, args=[self.authorizer, group])
+
+    def drop_group(self, group):
+        self.rest_client.delete(REQ_AUTHORIZATION_GROUP_MAPPING, args=[self.authorizer, group])
+
+    def groups(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_GROUP_MAPPINGS, args=[self.authorizer])
+
+    def group(self, group) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_GROUP_MAPPING, args=[self.authorizer, group])
+    
+    def roles(self):
+        return self.rest_client.get_json(REQ_AUTHORIZATION_ROLES, args=[self.authenticator])
+ 
+    def add_role(self, role):
+        self.rest_client.post(REQ_AUTHORIZATION_ROLE, None, args=[self.authenticator, role])
+   
+    def drop_role(self, role):
+        self.rest_client.delete(REQ_AUTHORIZATION_ROLE, args=[self.authorizer, role])
+
+    def set_role_permissions(self, role, permissions):
+        self.rest_client.post_only_json(REQ_AUTHORIZATION_ROLE_PERMISSIONS, permissions, args=[self.authenticator, role])
+
+    def role_permissions(self, role):
+        return self.rest_client.get_json(REQ_AUTHORIZATION_ROLE_PERMISSIONS, args=[self.authenticator, role])
+    
+    def assign_role_to_user(self, role, user):
+        self.rest_client.post(REQ_AUTHORIZATION_USER_ROLE, None, args=[self.authenticator, user, role])
+
+    def revoke_role_from_user(self, role, user):
+        self.rest_client.delete(REQ_AUTHORIZATION_USER_ROLE, args=[self.authenticator, user, role])
+
+    def assign_role_to_group(self, group, role):
+        self.rest_client.post(REQ_AUTHORIZATION_GROUP_ROLE, None, args=[self.authenticator, group, role])
+
+    def revoke_role_from_group(self, group, role):
+        self.rest_client.delete(REQ_AUTHORIZATION_GROUP_ROLE, args=[self.authenticator, group, role])
+
+    def user_map(self):
+        # Result uses Smile encoding, not JSON. This is really just for sanity
+        # checks: a Python client can't make use of the info.
+        # To decode, see newsmile: https://pypi.org/project/newsmile/
+        # However, the format Druid returns is not quite compatible with newsmile
+        return self.rest_client.get(REQ_USER_MAP, args=[self.authenticator])
+
+    # Convenience methods
+
+    def add_user(self, user, password):
+        '''
+        Adds a user to both the authenticator and authorizer.
+        '''
+        self.create_authentication_user(user)
+        self.set_password(user, password)
+        self.create_authorization_user(user)
+
+    def drop_user(self, user):
+        '''
+        Drops a user from both the authenticator and authorizer.
+        '''
+        self.drop_authorization_user(user)
+        self.drop_authentication_user(user)
+
+    def users(self):
+        '''
+        Returns the list of authenticator and authorizer users.
+        '''
+        return {
+            "authenticator": self.authorization_users(),
+            "authorizer": self.authentication_users()
+        }
+
+    def status(self):
+        '''
+        Returns both the authenticator and authorizer status.
+        '''
+        return {
+            "authenticator": self.authorization_status(),

Review Comment:
   Same as below - I think the status assignment for authenticator and authorizer is flipped.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] paul-rogers commented on pull request #14009: Add basic security functions to druidapi

Posted by "paul-rogers (via GitHub)" <gi...@apache.org>.
paul-rogers commented on PR #14009:
URL: https://github.com/apache/druid/pull/14009#issuecomment-1496776013

   The build is clean except for two issues:
   
   * The "slow" Kafka tests which repeatedly fail in many PRs.
   * The code scanning task which has started failing due to a missing config file.
   
   If we ignore the above two issues, which are independent of the code in this PR, the build is clean.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekrb19 commented on a diff in pull request #14009: Add basic security functions to druidapi

Posted by "abhishekrb19 (via GitHub)" <gi...@apache.org>.
abhishekrb19 commented on code in PR #14009:
URL: https://github.com/apache/druid/pull/14009#discussion_r1160797075


##########
docs/operations/security-overview.md:
##########
@@ -129,24 +122,23 @@ Within Druid's operating context, authenticators control the way user identities
 
 The following graphic depicts the course of request through the authentication process:
 
-
-![Druid security check flow](../assets/security-model-1.png "Druid security check flow") 
-
+![Druid security check flow](../assets/security-model-1.png "Druid security check flow")
 
 ## Enable an authenticator
 
-To authenticate requests in Druid, you configure an Authenticator. Authenticator extensions exist for HTTP basic authentication, LDAP, and Kerberos.  
+To authenticate requests in Druid, you configure an Authenticator. Authenticator extensions exist for HTTP basic authentication, LDAP, and Kerberos.
 
-The following takes you through sample configuration steps for enabling basic auth:  
+The following takes you through sample configuration steps for enabling basic auth:
 
 1. Add the `druid-basic-security` extension to `druid.extensions.loadList` in `common.runtime.properties`. For the quickstart installation, for example, the properties file is at `conf/druid/cluster/_common`:
    ```
    druid.extensions.loadList=["druid-basic-security", "druid-histogram", "druid-datasketches", "druid-kafka-indexing-service"]
    ```
-2. Configure the basic Authenticator, Authorizer, and Escalator settings in the same common.runtime.properties file. The Escalator defines how Druid processes authenticate with one another. 
+2. Configure the basic Authenticator, Authorizer, and Escalator settings in the same common.runtime.properties file. The Escalator defines how Druid processes authenticate with one another.
 
-An example configuration:
-   ```
+   An example configuration:
+
+   ```text

Review Comment:
   nit: if you switch this from `text` -> `properties`, you get some nice syntax highlighting
   
   ```suggestion
      ```properties



##########
docs/operations/security-overview.md:
##########
@@ -176,70 +170,83 @@ An example configuration:
    druid.auth.authorizer.MyBasicMetadataAuthorizer.type=basic
    ```
 
-3. Restart the cluster. 
+3. Restart the cluster.
 
-See [Authentication and Authorization](../design/auth.md) for more information about the Authenticator, Escalator, and Authorizer concepts. See [Basic Security](../development/extensions-core/druid-basic-security.md) for more information about the extension used in the examples above, and [Kerberos](../development/extensions-core/druid-kerberos.md) for Kerberos authentication.
+See the following topics for more information:
 
+* [Authentication and Authorization](../design/auth.md) for more information about the Authenticator,
+Escalator, and Authorizer.
+* [Basic Security](../development/extensions-core/druid-basic-security.md) for more information about
+the extension used in the examples above.
+* [Kerberos](../development/extensions-core/druid-kerberos.md) for Kerberos authentication.
+* [User authentication and authorization](security-user-auth.md) for details about permissions.
+* [SQL permissions](security-user-auth.md#sql-permissions) for permissions on SQL system tables.
+* [The `druidapi` Python library](../tutorials/tutorial-jupyter-index.md),
+  provided as part of the Druid tutorials, for functions you can
+  use in a Jupyter notebook to learn how security works, and to set up users and roles for testing.
 
 ## Enable authorizers
 
-After enabling the basic auth extension, you can add users, roles, and permissions via the Druid Coordinator `user` endpoint. Note that you cannot assign permissions directly to individual users. They must be assigned through roles. 
+After enabling the basic auth extension, you can add users, roles, and permissions via the Druid Coordinator `user` endpoint. Note that you cannot assign permissions directly to individual users. They must be assigned through roles.
 
 The following diagram depicts the authorization model, and the relationship between users, roles, permissions, and resources.
- 
-![Druid Security model](../assets/security-model-2.png "Druid security model") 
 
+![Druid Security model](../assets/security-model-2.png "Druid security model")
 
-The following steps walk through a sample setup procedure:  
+
+The following steps walk through a sample setup procedure:
 
 > The default Coordinator API port is 8081 for non-TLS connections and 8281 for secured connections.
 
-1. Create a user by issuing a POST request to `druid-ext/basic-security/authentication/db/MyBasicMetadataAuthenticator/users/<USERNAME>`, replacing USERNAME with the *new* username you are trying to create. For example: 
-  ```
+1. Create a user by issuing a POST request to `druid-ext/basic-security/authentication/db/MyBasicMetadataAuthenticator/users/<USERNAME>`,
+   replacing USERNAME with the *new* username you are trying to create. For example:
+   ```bash
    curl -u admin:password1 -XPOST https://my-coordinator-ip:8281/druid-ext/basic-security/authentication/db/MyBasicMetadataAuthenticator/users/myname
-  ```
-  >  If you have TLS enabled, be sure to adjust the curl command accordingly. For example, if your Druid servers use self-signed certificates, you may choose to include the `insecure` curl option to forgo certificate checking for the curl command. 
+   ```
+   >  If you have TLS enabled, be sure to adjust the curl command accordingly. For example, if your Druid servers use self-signed certificates,
+   you may choose to include the `insecure` curl option to forgo certificate checking for the curl command.
+
 2. Add a credential for the user by issuing a POST to `druid-ext/basic-security/authentication/db/MyBasicMetadataAuthenticator/users/<USERNAME>/credentials`. For example:
-    ```
-    curl -u admin:password1 -H'Content-Type: application/json' -XPOST https://my-coordinator-ip:8281/druid-ext/basic-security/authentication/db/MyBasicMetadataAuthenticator/users/myname/credentials --data-raw '{"password": "my_password"}'
-    ```
-2. For each authenticator user you create, create a corresponding authorizer user by issuing a POST request to `druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/users/<USERNAME>`. For example: 
-	```
-	curl -u admin:password1 -XPOST https://my-coordinator-ip:8281/druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/users/myname
-	```
-3. Create authorizer roles to control permissions by issuing a POST request to `druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/roles/<ROLENAME>`. For example: 
-	```
+   ```bash
+   curl -u admin:password1 -H'Content-Type: application/json' -XPOST https://my-coordinator-ip:8281/druid-ext/basic-security/authentication/db/MyBasicMetadataAuthenticator/users/myname/credentials --data-raw '{"password": "my_password"}'
+   ```
+2. For each authenticator user you create, create a corresponding authorizer user by issuing a POST request to `druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/users/<USERNAME>`. For example:
+	 ```bash
+	 curl -u admin:password1 -XPOST https://my-coordinator-ip:8281/druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/users/myname
+	 ```
+3. Create authorizer roles to control permissions by issuing a POST request to `druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/roles/<ROLENAME>`. For example:
+	 ```bash
    curl -u admin:password1 -XPOST https://my-coordinator-ip:8281/druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/roles/myrole
    ```
-4. Assign roles to users by issuing a POST request to `druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/users/<USERNAME>/roles/<ROLENAME>`. For example: 
-	```
-	curl -u admin:password1 -XPOST https://my-coordinator-ip:8281/druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/users/myname/roles/myrole | jq
-	```
-5. Finally, attach permissions to the roles to control how they can interact with Druid at `druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/roles/<ROLENAME>/permissions`. 
-	For example: 
-	```
-	curl -u admin:password1 -H'Content-Type: application/json' -XPOST --data-binary @perms.json https://my-coordinator-ip:8281/druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/roles/myrole/permissions
-	```
+4. Assign roles to users by issuing a POST request to `druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/users/<USERNAME>/roles/<ROLENAME>`. For example:
+	 ```bash
+	 curl -u admin:password1 -XPOST https://my-coordinator-ip:8281/druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/users/myname/roles/myrole | jq
+	 ```
+5. Finally, attach permissions to the roles to control how they can interact with Druid at `druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/roles/<ROLENAME>/permissions`.
+	 For example:
+	 ```bash
+	 curl -u admin:password1 -H'Content-Type: application/json' -XPOST --data-binary @perms.json https://my-coordinator-ip:8281/druid-ext/basic-security/authorization/db/MyBasicMetadataAuthorizer/roles/myrole/permissions
+	 ```
 	The payload of `perms.json` should be in the form:
-   	```
-    [
+ 	```json
+  [
     {
       "resource": {
-        "name": "<PATTERN>",
-        "type": "DATASOURCE"
+        "type": "DATASOURCE",
+        "name": "<PATTERN>"
       },
       "action": "READ"
     },
     {
       "resource": {
-      "name": "STATE",
-      "type": "STATE"
-    },
-    "action": "READ"
+        "type": "STATE",
+        "name": "STATE"
+      },
+      "action": "READ"
     }
-    ]
-    ```
-    > Note: Druid treats the resource name as a regular expression (regex). You can use a specific datasource name or regex to grant permissions for multiple datasources at a time.
+  ]
+  ```

Review Comment:
   I suspect there's a missing backtick in this diff that's causing markdown to render the following section inside a codeblock - please see screenshot:
   <img width="1190" alt="CleanShot 2023-04-07 at 09 19 12@2x" src="https://user-images.githubusercontent.com/8687261/230642632-2cb849a2-e56d-4e8f-8cc5-26198376c04f.png">
   



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/basic_auth.py:
##########
@@ -0,0 +1,238 @@
+# 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.
+
+BASIC_AUTH_BASE = '/druid-ext/basic-security'
+
+AUTHENTICATION_BASE = BASIC_AUTH_BASE + '/authentication'
+REQ_AUTHENTICATION_LOAD_STATUS = AUTHENTICATION_BASE + '/loadStatus'
+REQ_AUTHENTICATION_REFRESH_ALL = AUTHENTICATION_BASE + '/refreshAll'
+AUTHENTICATOR_BASE = AUTHENTICATION_BASE + '/db/{}'
+REQ_AUTHENTICATION_USERS = AUTHENTICATOR_BASE + '/users'
+REQ_AUTHENTICATION_USER = REQ_AUTHENTICATION_USERS + '/{}'
+REQ_AUTHENTICATION_CREDENTIALS = REQ_AUTHENTICATION_USER + '/credentials'
+
+AUTHORIZATION_BASE = BASIC_AUTH_BASE + '/authorization'
+REQ_AUTHORIZATION_LOAD_STATUS = AUTHORIZATION_BASE + '/loadStatus'
+REQ_AUTHORIZATION_REFRESH_ALL = AUTHORIZATION_BASE + '/refreshAll'
+AUTHORIZATION_BASE = AUTHORIZATION_BASE + '/db/{}'
+REQ_AUTHORIZATION_USERS = AUTHORIZATION_BASE + '/users'
+REQ_AUTHORIZATION_USER = REQ_AUTHORIZATION_USERS + '/{}'
+REQ_AUTHORIZATION_USER_ROLES = REQ_AUTHORIZATION_USER + '/roles'
+REQ_AUTHORIZATION_USER_ROLE = REQ_AUTHORIZATION_USER_ROLES + '/{}'
+REQ_AUTHORIZATION_GROUP_MAPPINGS = AUTHORIZATION_BASE + '/groupMappings'
+REQ_AUTHORIZATION_GROUP_MAPPING = AUTHORIZATION_BASE + '/groupMappings/{}'
+REQ_AUTHORIZATION_GROUP_ROLES = REQ_AUTHORIZATION_GROUP_MAPPING + '/roles'
+REQ_AUTHORIZATION_GROUP_ROLE = REQ_AUTHORIZATION_GROUP_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLES = AUTHORIZATION_BASE + '/roles'
+REQ_AUTHORIZATION_ROLE = REQ_AUTHORIZATION_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLE_PERMISSIONS = REQ_AUTHORIZATION_ROLE + '/permissions'
+REQ_USER_MAP = AUTHORIZATION_BASE + '/cachedSerializedUserMap'
+
+class BasicAuthClient:
+    '''
+    Manage Basic security. The Druid session must be logged in with the super
+    user, or some other user who has permission to modify user credentials.
+
+    Each client works with one authorizer/authenticator pair. Create multiple clients if you have to
+    work with multiple authenticators on a single server.
+
+    The basic pattern to add users and permissions is:
+
+    ```
+    # Create a client for your coordinator (Basic auth is not proxied through the router)
+    coord = druidapi.jupyter_client('http://localhost:8081', auth=('admin', 'password'))
+
+    # Get a client for your authenticator and authorizer:
+    ac = coord.basic_security('yourAuthorizer', 'yourAuthenticator')
+
+    # Create a user in both the authenticator and authorizer
+    ac.add_user('bob', 'secret')
+
+    # Define a role
+    ac.add_role('myRole')
+
+    # Assign the role to the user
+    ac.assign_role_to_user('myRole', 'bob')
+
+    # Give the role some permissions
+    ac.grant_permissions('myRole', [[consts.DATASOURCE_RESOURCE, 'foo', consts.READ_ACTION]])
+    ```
+
+    Then use the various other methods to list users, roles and permissions to verify the
+    setup. You can then create a second Druid client that acts as the new user:
+
+    ```
+    bob_client = druidapi.jupyter_client('http://localhost:8888', auth=('bob', 'secret'))
+    ```
+
+    See https://druid.apache.org/docs/latest/operations/security-overview.html#enable-authorizers
+    '''
+
+    def __init__(self, rest_client, authenticator, authorizer=None):
+        self.rest_client = rest_client
+        self.authenticator = authenticator
+        self.authorizer = authorizer if authorizer else authenticator
+
+    # Authentication
+
+    def authentication_status(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_LOAD_STATUS)
+
+    def authentication_refresh(self) -> None:
+        self.rest_client.get(REQ_AUTHENTICATION_REFRESH_ALL)
+
+    def create_authentication_user(self, user) -> None:
+        self.rest_client.post(REQ_AUTHENTICATION_USER, None, args=[self.authenticator, user])
+
+    def set_password(self, user, password) -> None:
+        self.rest_client.post_only_json(REQ_AUTHENTICATION_CREDENTIALS, {'password': password}, args=[self.authenticator, user])
+
+    def drop_authentication_user(self, user) -> None:
+        self.rest_client.delete(REQ_AUTHENTICATION_USER, args=[self.authenticator, user])
+    
+    def authentication_user(self, user) -> dict:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_USER, args=[self.authenticator, user])
+    
+    def authentication_users(self) -> list:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_USERS, args=[self.authenticator])
+    
+    # Authorization
+    # Groups are not documented. Use at your own risk.
+
+    def authorization_status(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_LOAD_STATUS)
+
+    def authorization_refresh(self) -> None:
+        self.rest_client.get(REQ_AUTHORIZATION_REFRESH_ALL)
+
+    def create_authorization_user(self, user) -> None:
+        self.rest_client.post(REQ_AUTHORIZATION_USER, None, args=[self.authorizer, user])
+
+    def drop_authorization_user(self, user) -> None:
+        self.rest_client.delete(REQ_AUTHORIZATION_USER, args=[self.authenticator, user])
+    
+    def authorization_user(self, user) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_USER, args=[self.authorizer, user])
+    
+    def authorization_users(self) -> list:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_USERS, args=[self.authorizer])
+    
+    def create_group(self, group, payload):
+        self.rest_client.post_json(REQ_AUTHORIZATION_GROUP_MAPPING, payload, args=[self.authorizer, group])
+
+    def drop_group(self, group):
+        self.rest_client.delete(REQ_AUTHORIZATION_GROUP_MAPPING, args=[self.authorizer, group])
+
+    def groups(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_GROUP_MAPPINGS, args=[self.authorizer])
+
+    def group(self, group) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_GROUP_MAPPING, args=[self.authorizer, group])
+    
+    def roles(self):
+        return self.rest_client.get_json(REQ_AUTHORIZATION_ROLES, args=[self.authenticator])
+ 
+    def add_role(self, role):
+        self.rest_client.post(REQ_AUTHORIZATION_ROLE, None, args=[self.authenticator, role])
+   
+    def drop_role(self, role):
+        self.rest_client.delete(REQ_AUTHORIZATION_ROLE, args=[self.authorizer, role])
+
+    def set_role_permissions(self, role, permissions):
+        self.rest_client.post_only_json(REQ_AUTHORIZATION_ROLE_PERMISSIONS, permissions, args=[self.authenticator, role])
+
+    def role_permissions(self, role):
+        return self.rest_client.get_json(REQ_AUTHORIZATION_ROLE_PERMISSIONS, args=[self.authenticator, role])
+    
+    def assign_role_to_user(self, role, user):
+        self.rest_client.post(REQ_AUTHORIZATION_USER_ROLE, None, args=[self.authenticator, user, role])
+
+    def revoke_role_from_user(self, role, user):
+        self.rest_client.delete(REQ_AUTHORIZATION_USER_ROLE, args=[self.authenticator, user, role])
+
+    def assign_role_to_group(self, group, role):
+        self.rest_client.post(REQ_AUTHORIZATION_GROUP_ROLE, None, args=[self.authenticator, group, role])
+
+    def revoke_role_from_group(self, group, role):
+        self.rest_client.delete(REQ_AUTHORIZATION_GROUP_ROLE, args=[self.authenticator, group, role])
+
+    def user_map(self):
+        # Result uses Smile encoding, not JSON. This is really just for sanity
+        # checks: a Python client can't make use of the info.
+        # To decode, see newsmile: https://pypi.org/project/newsmile/
+        # However, the format Druid returns is not quite compatible with newsmile
+        return self.rest_client.get(REQ_USER_MAP, args=[self.authenticator])
+
+    # Convenience methods
+
+    def add_user(self, user, password):
+        '''
+        Adds a user to both the authenticator and authorizer.
+        '''
+        self.create_authentication_user(user)
+        self.set_password(user, password)
+        self.create_authorization_user(user)
+
+    def drop_user(self, user):
+        '''
+        Drops a user from both the authenticator and authorizer.
+        '''
+        self.drop_authorization_user(user)
+        self.drop_authentication_user(user)
+
+    def users(self):
+        '''
+        Returns the list of authenticator and authorizer users.
+        '''
+        return {
+            "authenticator": self.authorization_users(),
+            "authorizer": self.authentication_users()
+        }
+
+    def status(self):
+        '''
+        Returns both the authenticator and authorizer status.
+        '''
+        return {
+            "authenticator": self.authorization_status(),

Review Comment:
   Same as above - I think the status assignment for authenticator and authorizer is flipped.



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/basic_auth.py:
##########
@@ -0,0 +1,238 @@
+# 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.
+
+BASIC_AUTH_BASE = '/druid-ext/basic-security'
+
+AUTHENTICATION_BASE = BASIC_AUTH_BASE + '/authentication'
+REQ_AUTHENTICATION_LOAD_STATUS = AUTHENTICATION_BASE + '/loadStatus'
+REQ_AUTHENTICATION_REFRESH_ALL = AUTHENTICATION_BASE + '/refreshAll'
+AUTHENTICATOR_BASE = AUTHENTICATION_BASE + '/db/{}'
+REQ_AUTHENTICATION_USERS = AUTHENTICATOR_BASE + '/users'
+REQ_AUTHENTICATION_USER = REQ_AUTHENTICATION_USERS + '/{}'
+REQ_AUTHENTICATION_CREDENTIALS = REQ_AUTHENTICATION_USER + '/credentials'
+
+AUTHORIZATION_BASE = BASIC_AUTH_BASE + '/authorization'
+REQ_AUTHORIZATION_LOAD_STATUS = AUTHORIZATION_BASE + '/loadStatus'
+REQ_AUTHORIZATION_REFRESH_ALL = AUTHORIZATION_BASE + '/refreshAll'
+AUTHORIZATION_BASE = AUTHORIZATION_BASE + '/db/{}'
+REQ_AUTHORIZATION_USERS = AUTHORIZATION_BASE + '/users'
+REQ_AUTHORIZATION_USER = REQ_AUTHORIZATION_USERS + '/{}'
+REQ_AUTHORIZATION_USER_ROLES = REQ_AUTHORIZATION_USER + '/roles'
+REQ_AUTHORIZATION_USER_ROLE = REQ_AUTHORIZATION_USER_ROLES + '/{}'
+REQ_AUTHORIZATION_GROUP_MAPPINGS = AUTHORIZATION_BASE + '/groupMappings'
+REQ_AUTHORIZATION_GROUP_MAPPING = AUTHORIZATION_BASE + '/groupMappings/{}'
+REQ_AUTHORIZATION_GROUP_ROLES = REQ_AUTHORIZATION_GROUP_MAPPING + '/roles'
+REQ_AUTHORIZATION_GROUP_ROLE = REQ_AUTHORIZATION_GROUP_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLES = AUTHORIZATION_BASE + '/roles'
+REQ_AUTHORIZATION_ROLE = REQ_AUTHORIZATION_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLE_PERMISSIONS = REQ_AUTHORIZATION_ROLE + '/permissions'
+REQ_USER_MAP = AUTHORIZATION_BASE + '/cachedSerializedUserMap'
+
+class BasicAuthClient:
+    '''
+    Manage Basic security. The Druid session must be logged in with the super

Review Comment:
   nit: "Basic" -> "basic"



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/druid.py:
##########
@@ -116,6 +117,33 @@ def datasources(self) -> DatasourceClient:
         if not self.datasource_client:
             self.datasource_client = DatasourceClient(self.rest_client)
         return self.datasource_client
+    
+    def basic_security(self, authenticator, authorizer=None):
+        '''
+        Returns a client to work with a basic authorization authenticator/authorizer pair.
+        This client assumes the typical case of one authenticator and one authorizer. If
+        you have more than one, create multiple clients.
+
+        The basic security API is not proxied through the Router: it must work directly with
+        the Coordinator. Create an ad hoc Druid client for your Coordinator. Because you have
+        basic security enabled, you must specify the admin user and password:
+
+        ```
+        coord = druidapi.jupyter_client('http://localhost:8081', auth=('admin', 'admin-pwd'))

Review Comment:
   Neat!



##########
examples/quickstart/jupyter-notebooks/druidapi/druidapi/basic_auth.py:
##########
@@ -0,0 +1,238 @@
+# 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.
+
+BASIC_AUTH_BASE = '/druid-ext/basic-security'
+
+AUTHENTICATION_BASE = BASIC_AUTH_BASE + '/authentication'
+REQ_AUTHENTICATION_LOAD_STATUS = AUTHENTICATION_BASE + '/loadStatus'
+REQ_AUTHENTICATION_REFRESH_ALL = AUTHENTICATION_BASE + '/refreshAll'
+AUTHENTICATOR_BASE = AUTHENTICATION_BASE + '/db/{}'
+REQ_AUTHENTICATION_USERS = AUTHENTICATOR_BASE + '/users'
+REQ_AUTHENTICATION_USER = REQ_AUTHENTICATION_USERS + '/{}'
+REQ_AUTHENTICATION_CREDENTIALS = REQ_AUTHENTICATION_USER + '/credentials'
+
+AUTHORIZATION_BASE = BASIC_AUTH_BASE + '/authorization'
+REQ_AUTHORIZATION_LOAD_STATUS = AUTHORIZATION_BASE + '/loadStatus'
+REQ_AUTHORIZATION_REFRESH_ALL = AUTHORIZATION_BASE + '/refreshAll'
+AUTHORIZATION_BASE = AUTHORIZATION_BASE + '/db/{}'
+REQ_AUTHORIZATION_USERS = AUTHORIZATION_BASE + '/users'
+REQ_AUTHORIZATION_USER = REQ_AUTHORIZATION_USERS + '/{}'
+REQ_AUTHORIZATION_USER_ROLES = REQ_AUTHORIZATION_USER + '/roles'
+REQ_AUTHORIZATION_USER_ROLE = REQ_AUTHORIZATION_USER_ROLES + '/{}'
+REQ_AUTHORIZATION_GROUP_MAPPINGS = AUTHORIZATION_BASE + '/groupMappings'
+REQ_AUTHORIZATION_GROUP_MAPPING = AUTHORIZATION_BASE + '/groupMappings/{}'
+REQ_AUTHORIZATION_GROUP_ROLES = REQ_AUTHORIZATION_GROUP_MAPPING + '/roles'
+REQ_AUTHORIZATION_GROUP_ROLE = REQ_AUTHORIZATION_GROUP_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLES = AUTHORIZATION_BASE + '/roles'
+REQ_AUTHORIZATION_ROLE = REQ_AUTHORIZATION_ROLES + '/{}'
+REQ_AUTHORIZATION_ROLE_PERMISSIONS = REQ_AUTHORIZATION_ROLE + '/permissions'
+REQ_USER_MAP = AUTHORIZATION_BASE + '/cachedSerializedUserMap'
+
+class BasicAuthClient:
+    '''
+    Manage Basic security. The Druid session must be logged in with the super
+    user, or some other user who has permission to modify user credentials.
+
+    Each client works with one authorizer/authenticator pair. Create multiple clients if you have to
+    work with multiple authenticators on a single server.
+
+    The basic pattern to add users and permissions is:
+
+    ```
+    # Create a client for your coordinator (Basic auth is not proxied through the router)
+    coord = druidapi.jupyter_client('http://localhost:8081', auth=('admin', 'password'))
+
+    # Get a client for your authenticator and authorizer:
+    ac = coord.basic_security('yourAuthorizer', 'yourAuthenticator')
+
+    # Create a user in both the authenticator and authorizer
+    ac.add_user('bob', 'secret')
+
+    # Define a role
+    ac.add_role('myRole')
+
+    # Assign the role to the user
+    ac.assign_role_to_user('myRole', 'bob')
+
+    # Give the role some permissions
+    ac.grant_permissions('myRole', [[consts.DATASOURCE_RESOURCE, 'foo', consts.READ_ACTION]])
+    ```
+
+    Then use the various other methods to list users, roles and permissions to verify the
+    setup. You can then create a second Druid client that acts as the new user:
+
+    ```
+    bob_client = druidapi.jupyter_client('http://localhost:8888', auth=('bob', 'secret'))
+    ```
+
+    See https://druid.apache.org/docs/latest/operations/security-overview.html#enable-authorizers
+    '''
+
+    def __init__(self, rest_client, authenticator, authorizer=None):
+        self.rest_client = rest_client
+        self.authenticator = authenticator
+        self.authorizer = authorizer if authorizer else authenticator
+
+    # Authentication
+
+    def authentication_status(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_LOAD_STATUS)
+
+    def authentication_refresh(self) -> None:
+        self.rest_client.get(REQ_AUTHENTICATION_REFRESH_ALL)
+
+    def create_authentication_user(self, user) -> None:
+        self.rest_client.post(REQ_AUTHENTICATION_USER, None, args=[self.authenticator, user])
+
+    def set_password(self, user, password) -> None:
+        self.rest_client.post_only_json(REQ_AUTHENTICATION_CREDENTIALS, {'password': password}, args=[self.authenticator, user])
+
+    def drop_authentication_user(self, user) -> None:
+        self.rest_client.delete(REQ_AUTHENTICATION_USER, args=[self.authenticator, user])
+    
+    def authentication_user(self, user) -> dict:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_USER, args=[self.authenticator, user])
+    
+    def authentication_users(self) -> list:
+        return self.rest_client.get_json(REQ_AUTHENTICATION_USERS, args=[self.authenticator])
+    
+    # Authorization
+    # Groups are not documented. Use at your own risk.
+
+    def authorization_status(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_LOAD_STATUS)
+
+    def authorization_refresh(self) -> None:
+        self.rest_client.get(REQ_AUTHORIZATION_REFRESH_ALL)
+
+    def create_authorization_user(self, user) -> None:
+        self.rest_client.post(REQ_AUTHORIZATION_USER, None, args=[self.authorizer, user])
+
+    def drop_authorization_user(self, user) -> None:
+        self.rest_client.delete(REQ_AUTHORIZATION_USER, args=[self.authenticator, user])
+    
+    def authorization_user(self, user) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_USER, args=[self.authorizer, user])
+    
+    def authorization_users(self) -> list:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_USERS, args=[self.authorizer])
+    
+    def create_group(self, group, payload):
+        self.rest_client.post_json(REQ_AUTHORIZATION_GROUP_MAPPING, payload, args=[self.authorizer, group])
+
+    def drop_group(self, group):
+        self.rest_client.delete(REQ_AUTHORIZATION_GROUP_MAPPING, args=[self.authorizer, group])
+
+    def groups(self) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_GROUP_MAPPINGS, args=[self.authorizer])
+
+    def group(self, group) -> dict:
+        return self.rest_client.get_json(REQ_AUTHORIZATION_GROUP_MAPPING, args=[self.authorizer, group])
+    
+    def roles(self):
+        return self.rest_client.get_json(REQ_AUTHORIZATION_ROLES, args=[self.authenticator])
+ 
+    def add_role(self, role):
+        self.rest_client.post(REQ_AUTHORIZATION_ROLE, None, args=[self.authenticator, role])
+   
+    def drop_role(self, role):
+        self.rest_client.delete(REQ_AUTHORIZATION_ROLE, args=[self.authorizer, role])
+
+    def set_role_permissions(self, role, permissions):
+        self.rest_client.post_only_json(REQ_AUTHORIZATION_ROLE_PERMISSIONS, permissions, args=[self.authenticator, role])
+
+    def role_permissions(self, role):
+        return self.rest_client.get_json(REQ_AUTHORIZATION_ROLE_PERMISSIONS, args=[self.authenticator, role])
+    
+    def assign_role_to_user(self, role, user):
+        self.rest_client.post(REQ_AUTHORIZATION_USER_ROLE, None, args=[self.authenticator, user, role])
+
+    def revoke_role_from_user(self, role, user):
+        self.rest_client.delete(REQ_AUTHORIZATION_USER_ROLE, args=[self.authenticator, user, role])
+
+    def assign_role_to_group(self, group, role):
+        self.rest_client.post(REQ_AUTHORIZATION_GROUP_ROLE, None, args=[self.authenticator, group, role])
+
+    def revoke_role_from_group(self, group, role):
+        self.rest_client.delete(REQ_AUTHORIZATION_GROUP_ROLE, args=[self.authenticator, group, role])
+
+    def user_map(self):
+        # Result uses Smile encoding, not JSON. This is really just for sanity
+        # checks: a Python client can't make use of the info.
+        # To decode, see newsmile: https://pypi.org/project/newsmile/
+        # However, the format Druid returns is not quite compatible with newsmile
+        return self.rest_client.get(REQ_USER_MAP, args=[self.authenticator])
+
+    # Convenience methods
+
+    def add_user(self, user, password):
+        '''
+        Adds a user to both the authenticator and authorizer.
+        '''
+        self.create_authentication_user(user)
+        self.set_password(user, password)
+        self.create_authorization_user(user)
+
+    def drop_user(self, user):
+        '''
+        Drops a user from both the authenticator and authorizer.
+        '''
+        self.drop_authorization_user(user)
+        self.drop_authentication_user(user)
+
+    def users(self):
+        '''
+        Returns the list of authenticator and authorizer users.
+        '''
+        return {
+            "authenticator": self.authorization_users(),

Review Comment:
   Should this be:
   ```python
           return {
               "authenticator": self.authentication_users(),
               "authorizer": self. authorization_users()
           }



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] abhishekrb19 commented on pull request #14009: Add basic security functions to druidapi

Posted by "abhishekrb19 (via GitHub)" <gi...@apache.org>.
abhishekrb19 commented on PR #14009:
URL: https://github.com/apache/druid/pull/14009#issuecomment-1503848943

   This change was included in https://github.com/apache/druid/pull/14055


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] vtlim closed pull request #14009: Add basic security functions to druidapi

Posted by "vtlim (via GitHub)" <gi...@apache.org>.
vtlim closed pull request #14009: Add basic security functions to druidapi
URL: https://github.com/apache/druid/pull/14009


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org