getting-started/spark/notebooks/SparkPolaris.ipynb (804 lines of code) (raw):

{ "cells": [ { "cell_type": "markdown", "id": "f4ab2c33-c072-49e9-93de-da24759113f7", "metadata": {}, "source": [ "# Bootstrap the client with ROOT credentials\n", "Using the python client generated from our OpenAPI spec, we generate a token from our root user's credentials" ] }, { "cell_type": "code", "execution_count": null, "id": "f982815a-2b48-46ab-96a6-20dad7ec1420", "metadata": {}, "outputs": [], "source": [ "from polaris.catalog.api.iceberg_catalog_api import IcebergCatalogAPI\n", "from polaris.catalog.api.iceberg_o_auth2_api import IcebergOAuth2API\n", "from polaris.catalog.api_client import ApiClient as CatalogApiClient\n", "from polaris.catalog.api_client import Configuration as CatalogApiClientConfiguration\n", "\n", "polaris_credential = 'root:s3cr3t' # pragma: allowlist secret\n", "\n", "client_id, client_secret = polaris_credential.split(\":\")\n", "client = CatalogApiClient(CatalogApiClientConfiguration(username=client_id,\n", " password=client_secret,\n", " host='http://polaris:8181/api/catalog'))\n", "\n", "oauth_api = IcebergOAuth2API(client)\n", "token = oauth_api.get_token(scope='PRINCIPAL_ROLE:ALL',\n", " client_id=client_id,\n", " client_secret=client_secret,\n", " grant_type='client_credentials',\n", " _headers={'realm': 'default-realm'})\n" ] }, { "cell_type": "markdown", "id": "4c21f4a1-4129-4dd8-9a6c-fa6eeabfa56e", "metadata": {}, "source": [ "# Create our first catalog\n", "\n", "* Creates a catalog named `polaris_catalog` that writes to a specified location in the Local Filesystem." ] }, { "cell_type": "code", "execution_count": null, "id": "0f7a311a-9a55-4ff7-a40e-db3c74c53b9b", "metadata": {}, "outputs": [], "source": [ "from polaris.management import *\n", "\n", "client = ApiClient(Configuration(access_token=token.access_token,\n", " host='http://polaris:8181/api/management/v1'))\n", "root_client = PolarisDefaultApi(client)\n", "\n", "storage_conf = FileStorageConfigInfo(storage_type=\"FILE\", allowed_locations=[\"file:///tmp\"])\n", "catalog_name = 'polaris_demo'\n", "catalog = Catalog(name=catalog_name, type='INTERNAL', properties={\"default-base-location\": \"file:///tmp/polaris/\"},\n", " storage_config_info=storage_conf)\n", "catalog.storage_config_info = storage_conf\n", "root_client.create_catalog(create_catalog_request=CreateCatalogRequest(catalog=catalog))\n", "resp = root_client.get_catalog(catalog_name=catalog.name)\n", "resp" ] }, { "cell_type": "markdown", "id": "6521039f-c25d-4baa-96ae-a4408c0fced0", "metadata": {}, "source": [ "# Utility Functions" ] }, { "cell_type": "code", "execution_count": null, "id": "e3e42c12-4e01-4577-bdf5-90c2704a5de8", "metadata": {}, "outputs": [], "source": [ "# Creates a principal with the given name\n", "def create_principal(api, principal_name):\n", " principal = Principal(name=principal_name, type=\"SERVICE\")\n", " try:\n", " principal_result = api.create_principal(CreatePrincipalRequest(principal=principal))\n", " return principal_result\n", " except ApiException as e:\n", " if e.status == 409:\n", " return api.rotate_credentials(principal_name=principal_name)\n", " else:\n", " raise e\n", "\n", "# Create a catalog role with the given name\n", "def create_catalog_role(api, catalog, role_name):\n", " catalog_role = CatalogRole(name=role_name)\n", " try:\n", " api.create_catalog_role(catalog_name=catalog.name, create_catalog_role_request=CreateCatalogRoleRequest(catalog_role=catalog_role))\n", " return api.get_catalog_role(catalog_name=catalog.name, catalog_role_name=role_name)\n", " except ApiException as e:\n", " return api.get_catalog_role(catalog_name=catalog.name, catalog_role_name=role_name)\n", " else:\n", " raise e\n", "\n", "# Create a principal role with the given name\n", "def create_principal_role(api, role_name):\n", " principal_role = PrincipalRole(name=role_name)\n", " try:\n", " api.create_principal_role(CreatePrincipalRoleRequest(principal_role=principal_role))\n", " return api.get_principal_role(principal_role_name=role_name)\n", " except ApiException as e:\n", " return api.get_principal_role(principal_role_name=role_name)\n" ] }, { "cell_type": "markdown", "id": "15c250ca-7161-418e-bc52-8bbd88a3e57c", "metadata": {}, "source": [ "# Create a new Principal, Principal Role, and Catalog Role\n", "The new Principal belongs to the `engineer` principal role, which has `CATALOG_MANAGE_CONTENT` privileges on the `polaris_catalog`. \n", "\n", "\n", "`CATALOG_MANAGE_CONTENT` has create/list/read/write privileges on all entities within the catalog. The same privilege could be granted to a namespace, in which case, the engineers could create/list/read/write any entity under that namespace" ] }, { "cell_type": "code", "execution_count": null, "id": "c5ceb5ca-f977-46c7-b2a6-07dda59e8a8b", "metadata": {}, "outputs": [], "source": [ "# Create the engineer_principal\n", "engineer_principal = create_principal(root_client, \"collado\")\n", "\n", "# Create the principal role\n", "engineer_role = create_principal_role(root_client, \"engineer\")\n", "\n", "# Create the catalog role\n", "manager_catalog_role = create_catalog_role(root_client, catalog, \"manage_catalog\")\n", "\n", "# Grant the catalog role to the principal role\n", "# All principals in the principal role have the catalog role's privileges\n", "root_client.assign_catalog_role_to_principal_role(principal_role_name=engineer_role.name,\n", " catalog_name=catalog.name,\n", " grant_catalog_role_request=GrantCatalogRoleRequest(catalog_role=manager_catalog_role))\n", "\n", "# Assign privileges to the catalog role\n", "# Here, we grant CATALOG_MANAGE_CONTENT\n", "root_client.add_grant_to_catalog_role(catalog.name, manager_catalog_role.name,\n", " AddGrantRequest(grant=CatalogGrant(catalog_name=catalog.name,\n", " type='catalog',\n", " privilege=CatalogPrivilege.CATALOG_MANAGE_CONTENT)))\n", "\n", "# Assign the principal role to the principal\n", "root_client.assign_principal_role(engineer_principal.principal.name, grant_principal_role_request=GrantPrincipalRoleRequest(principal_role=engineer_role))" ] }, { "cell_type": "markdown", "id": "8a04cf15-a327-4ab9-a083-6da4e7dd1623", "metadata": {}, "source": [ "# Create a reader Principal, Principal Role, and Catalog Role\n", "This new principal belongs to the `product_manager` principal role, which is explicitly granted read and list permissions on the catalog.\n", "\n", "Permissions cascade, so permissions granted at the catalog level are inherited by namespaces and tables within the catalog." ] }, { "cell_type": "code", "execution_count": null, "id": "b51a6433-99c9-46c5-a855-928e30bad6e5", "metadata": {}, "outputs": [], "source": [ "# Create a reader principal\n", "reader_principal = create_principal(root_client, \"mlee\")\n", "\n", "# Create the principal role\n", "pm_role = create_principal_role(root_client, \"product_manager\")\n", "\n", "# Create the catalog role\n", "read_only_role = create_catalog_role(root_client, catalog, \"read_only\")\n", "\n", "# Grant the catalog role to the principal role\n", "root_client.assign_catalog_role_to_principal_role(principal_role_name=pm_role.name,\n", " catalog_name=catalog.name,\n", " grant_catalog_role_request=GrantCatalogRoleRequest(catalog_role=read_only_role))\n", "\n", "# Assign privileges to the catalog role\n", "# Here, the catalog role is granted READ and LIST privileges at the catalog level\n", "# Privileges cascade down\n", "root_client.add_grant_to_catalog_role(catalog.name, read_only_role.name,\n", " AddGrantRequest(grant=CatalogGrant(catalog_name=catalog.name,\n", " type='catalog',\n", " privilege=CatalogPrivilege.TABLE_LIST)))\n", "root_client.add_grant_to_catalog_role(catalog.name, read_only_role.name,\n", " AddGrantRequest(grant=CatalogGrant(catalog_name=catalog.name,\n", " type='catalog',\n", " privilege=CatalogPrivilege.TABLE_READ_PROPERTIES)))\n", "root_client.add_grant_to_catalog_role(catalog.name, read_only_role.name,\n", " AddGrantRequest(grant=CatalogGrant(catalog_name=catalog.name,\n", " type='catalog',\n", " privilege=CatalogPrivilege.TABLE_READ_DATA)))\n", "root_client.add_grant_to_catalog_role(catalog.name, read_only_role.name,\n", " AddGrantRequest(grant=CatalogGrant(catalog_name=catalog.name,\n", " type='catalog',\n", " privilege=CatalogPrivilege.VIEW_LIST)))\n", "root_client.add_grant_to_catalog_role(catalog.name, read_only_role.name,\n", " AddGrantRequest(grant=CatalogGrant(catalog_name=catalog.name,\n", " type='catalog',\n", " privilege=CatalogPrivilege.VIEW_READ_PROPERTIES)))\n", "root_client.add_grant_to_catalog_role(catalog.name, read_only_role.name,\n", " AddGrantRequest(grant=CatalogGrant(catalog_name=catalog.name,\n", " type='catalog',\n", " privilege=CatalogPrivilege.NAMESPACE_READ_PROPERTIES)))\n", "root_client.add_grant_to_catalog_role(catalog.name, read_only_role.name,\n", " AddGrantRequest(grant=CatalogGrant(catalog_name=catalog.name,\n", " type='catalog',\n", " privilege=CatalogPrivilege.NAMESPACE_LIST)))\n", "\n", "# Assign the principal role to the principal\n", "root_client.assign_principal_role(reader_principal.principal.name, grant_principal_role_request=GrantPrincipalRoleRequest(principal_role=pm_role))" ] }, { "cell_type": "markdown", "id": "14c1e2b3-a0d4-49b5-8e1e-ddb43f98b115", "metadata": {}, "source": [ "# Create a Spark session with the engineer credentials\n", "\n", "* Catalog URI points to our Polaris installation\n", "* Credential set using the client_id and client_secret generated for the principal\n", "* Scope set to `PRINCIPAL_ROLE:ALL`\n", "* `X-Iceberg-Access-Delegation` is set to vended-credentials" ] }, { "cell_type": "code", "execution_count": null, "id": "fd13f24b-9d59-470d-9be1-660c22dde680", "metadata": { "tags": [] }, "outputs": [], "source": [ "from pyspark.sql import SparkSession\n", "\n", "spark = (SparkSession.builder\n", " .config(\"spark.sql.catalog.spark_catalog\", \"org.apache.iceberg.spark.SparkSessionCatalog\")\n", " .config(\"spark.jars.packages\", \"org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:1.9.0,org.apache.hadoop:hadoop-aws:3.4.0,software.amazon.awssdk:bundle:2.23.19,software.amazon.awssdk:url-connection-client:2.23.19\")\n", " .config('spark.sql.iceberg.vectorization.enabled', 'false')\n", " \n", " # Configure the 'polaris' catalog as an Iceberg rest catalog\n", " .config(\"spark.sql.catalog.polaris.type\", \"rest\")\n", " .config(\"spark.sql.catalog.polaris\", \"org.apache.iceberg.spark.SparkCatalog\")\n", " # Specify the rest catalog endpoint \n", " .config(\"spark.sql.catalog.polaris.uri\", \"http://polaris:8181/api/catalog\")\n", " # Enable token refresh\n", " .config(\"spark.sql.catalog.polaris.token-refresh-enabled\", \"true\")\n", " # specify the client_id:client_secret pair\n", " .config(\"spark.sql.catalog.polaris.credential\", f\"{engineer_principal.credentials.client_id}:{engineer_principal.credentials.client_secret}\")\n", "\n", " # Set the warehouse to the name of the catalog we created\n", " .config(\"spark.sql.catalog.polaris.warehouse\", catalog_name)\n", "\n", " # Scope set to PRINCIPAL_ROLE:ALL\n", " .config(\"spark.sql.catalog.polaris.scope\", 'PRINCIPAL_ROLE:ALL')\n", "\n", " # Enable access credential delegation\n", " .config(\"spark.sql.catalog.polaris.header.X-Iceberg-Access-Delegation\", 'vended-credentials')\n", "\n", " .config(\"spark.sql.catalog.polaris.io-impl\", \"org.apache.iceberg.io.ResolvingFileIO\")\n", " .config(\"spark.sql.catalog.polaris.s3.region\", \"us-west-2\")\n", " .config(\"spark.history.fs.logDirectory\", \"/home/iceberg/spark-events\")).getOrCreate()\n" ] }, { "cell_type": "markdown", "id": "f1cfef99-4a52-433b-ac1d-c92db5f396a3", "metadata": {}, "source": [ "# USE polaris\n", "Tell Spark to use the Polaris catalog" ] }, { "cell_type": "code", "execution_count": null, "id": "72e9e5fb-b22e-4d38-bb1e-4ca78c0d0f3e", "metadata": { "tags": [] }, "outputs": [], "source": [ "spark.sql(\"USE polaris\")\n", "spark.sql(\"SHOW NAMESPACES\").show()" ] }, { "cell_type": "markdown", "id": "6b6c5a4a-d469-4364-9249-1a4aeb4d560c", "metadata": {}, "source": [ "# Create Nested Namespaces" ] }, { "cell_type": "code", "execution_count": null, "id": "54159ab2-5964-49a0-8202-a4b64ee4f9e7", "metadata": { "tags": [] }, "outputs": [], "source": [ "spark.sql(\"CREATE NAMESPACE IF NOT EXISTS COLLADO_TEST\")\n", "spark.sql(\"CREATE NAMESPACE IF NOT EXISTS COLLADO_TEST.PUBLIC\")\n", "spark.sql(\"SHOW NAMESPACES IN COLLADO_TEST\").show()" ] }, { "cell_type": "markdown", "id": "51a5311e-4a40-4bdc-aaee-b5845e06d020", "metadata": {}, "source": [ "# Create a table" ] }, { "cell_type": "code", "execution_count": null, "id": "4abc8426-7f2a-4f3f-9e26-1f1824f870c6", "metadata": { "tags": [] }, "outputs": [], "source": [ "spark.sql(\"USE NAMESPACE COLLADO_TEST.PUBLIC\")\n", "spark.sql(\"\"\"CREATE TABLE IF NOT EXISTS TEST_TABLE (\n", " id bigint NOT NULL COMMENT 'unique id',\n", " data string)\n", "USING iceberg;\n", "\"\"\")" ] }, { "cell_type": "markdown", "id": "91fa7c6c-34e0-4bb9-babc-3f3db4778101", "metadata": {}, "source": [ "# It's Empty" ] }, { "cell_type": "code", "execution_count": null, "id": "ff5a466d-6a67-4f42-a6a6-ac54ec258e54", "metadata": { "tags": [] }, "outputs": [], "source": [ "spark.sql(\"SELECT * FROM TEST_TABLE\").show()" ] }, { "cell_type": "markdown", "id": "f7b297de-ed11-41df-8ed9-e9396c6c4465", "metadata": {}, "source": [ "# Insert some records\n", "Querying again shows some records" ] }, { "cell_type": "code", "execution_count": null, "id": "d7ab2991-6de9-4105-9f95-4c9f1c18f426", "metadata": { "tags": [] }, "outputs": [], "source": [ "spark.sql(\"INSERT INTO TEST_TABLE VALUES (1, 'some data'), (2, 'more data'), (3, 'yet more data')\")\n", "spark.sql(\"SELECT * FROM TEST_TABLE\").show()" ] }, { "cell_type": "markdown", "id": "831064fe-d3fe-4478-9604-3a1eb91a33d4", "metadata": {}, "source": [ "# Use the Catalog API client\n", "Create a new client using the engineer credentials" ] }, { "cell_type": "code", "execution_count": null, "id": "75131c37-2ad2-4e6d-bd65-64a915bb6694", "metadata": {}, "outputs": [], "source": [ "# Create a client to fetch an API token - use our client_id and client_secret as the username/password\n", "token_client = CatalogApiClient(CatalogApiClientConfiguration(username=engineer_principal.credentials.client_id,\n", " password=engineer_principal.credentials.client_secret,\n", " host='http://polaris:8181/api/catalog'))\n", "\n", "# Use the client to get the token from the /tokens endpoint\n", "collado_token = IcebergOAuth2API(token_client).get_token(scope='PRINCIPAL_ROLE:ALL',\n", " client_id=engineer_principal.credentials.client_id,\n", " client_secret=engineer_principal.credentials.client_secret,\n", " grant_type='client_credentials',\n", " _headers={'realm': 'default-realm'})\n", "\n", "# Now create a catalog client that uses the token in its Authentication header\n", "client = CatalogApiClient(CatalogApiClientConfiguration(access_token=collado_token.access_token,\n", " host='http://polaris:8181/api/catalog'))\n", "collado_client = IcebergCatalogAPI(client)\n" ] }, { "cell_type": "markdown", "id": "485595fc-ea26-45c8-bd6b-4c7131cea834", "metadata": {}, "source": [ "# Directly call the LoadTable API\n", "This is the response returned to Spark. In addition to the expected metadataLocation, snaphshot, and schema data, we also see S3 credentials in the `config` field" ] }, { "cell_type": "code", "execution_count": null, "id": "cf24575d-4bfd-456c-962c-82f80fda5cc0", "metadata": {}, "outputs": [], "source": [ "import codecs\n", "import json\n", "from IPython.display import display, JSON\n", "\n", "def format_namespace(namespace):\n", " return codecs.decode(\"1F\", \"hex\").decode(\"UTF-8\").join(namespace)\n", "\n", "# Call loadTable\n", "tbl_meta = collado_client.load_table(prefix=catalog_name, namespace=format_namespace(['COLLADO_TEST', 'PUBLIC']), table='TEST_TABLE', x_iceberg_access_delegation='true')\n", "display(JSON(tbl_meta.to_dict(), expanded=True))" ] }, { "cell_type": "markdown", "id": "2a76a0ba-800b-436c-b617-3725286af58c", "metadata": {}, "source": [ "# Initiate a new Spark session\n", "Change the credentials to the PM's read-only credentials" ] }, { "cell_type": "code", "execution_count": null, "id": "6f3aac79-bf45-4603-bd64-30eeab4bdfa7", "metadata": {}, "outputs": [], "source": [ "# The new spark session inherits everything from the previous session except for the overridden credentials\n", "new_spark = spark.newSession()\n", "new_spark.conf.set(\"spark.sql.catalog.polaris.credential\", f\"{reader_principal.credentials.client_id}:{reader_principal.credentials.client_secret}\")\n", "new_spark.sql(\"USE polaris\")" ] }, { "cell_type": "markdown", "id": "b6ba9acb-2e9d-4ffa-a685-8a85c75f3046", "metadata": {}, "source": [ "# Show Namespace contents\n", "We can still `USE NAMESPACE` and `SHOW TABLES`, which require `READ_NAMESPACE_PROPERTIES` and `LIST_TABLES` privileges respectively" ] }, { "cell_type": "code", "execution_count": null, "id": "d517424d-8893-4375-ac3b-c532c8682b6a", "metadata": {}, "outputs": [], "source": [ "new_spark.sql(\"USE NAMESPACE COLLADO_TEST.PUBLIC\")\n", "new_spark.sql(\"SHOW TABLES\").show()" ] }, { "cell_type": "markdown", "id": "ecfba50e-ec5d-41dd-8715-78ea1c1f42e2", "metadata": {}, "source": [ "# Table reads work" ] }, { "cell_type": "code", "execution_count": null, "id": "7fce4b1f-4d71-4d03-8b60-3e9ca6ca6ddf", "metadata": {}, "outputs": [], "source": [ "new_spark.sql(\"SELECT * FROM TEST_TABLE\").show()" ] }, { "cell_type": "markdown", "id": "1c79b535-989a-43d1-a7ef-7ccca574c0db", "metadata": {}, "source": [ "# Insert attempts will fail" ] }, { "cell_type": "code", "execution_count": null, "id": "27434b46-1d40-4bd5-b247-66ec069db265", "metadata": { "editable": true, "slideshow": { "slide_type": "" }, "tags": [ "raises-exception" ] }, "outputs": [], "source": [ "new_spark.sql(\"INSERT INTO TEST_TABLE VALUES (4, 'you cannot see this data'), (5, 'it will never be inserted'), (6, 'sad emoji')\")" ] }, { "cell_type": "markdown", "id": "430d5262-9678-4bb1-ac0d-1d4cc3ca5116", "metadata": {}, "source": [ "# Create an API client using reader credentials" ] }, { "cell_type": "code", "execution_count": null, "id": "1386becf-fa56-4c05-8475-b2562aae9c05", "metadata": { "editable": true, "slideshow": { "slide_type": "" }, "tags": [] }, "outputs": [], "source": [ "# Create a client to fetch an API token - use the reader's client_id and client_secret as the username/password\n", "token_client = CatalogApiClient(CatalogApiClientConfiguration(username=reader_principal.credentials.client_id,\n", " password=reader_principal.credentials.client_secret,\n", " host='http://polaris:8181/api/catalog'))\n", "\n", "# Get the token\n", "pm_token = IcebergOAuth2API(token_client).get_token(scope='PRINCIPAL_ROLE:ALL',\n", " client_id=reader_principal.credentials.client_id,\n", " client_secret=reader_principal.credentials.client_secret,\n", " grant_type='client_credentials',\n", " _headers={'realm': 'default-realm'})\n", "\n", "# Now create a catalog client that uses the token in its Authentication header\n", "pm_client = IcebergCatalogAPI(CatalogApiClient(CatalogApiClientConfiguration(access_token=pm_token.access_token,\n", " host='http://polaris:8181/api/catalog')))\n" ] }, { "cell_type": "markdown", "id": "b699317e-c5e2-4f8b-b99c-9131f3d36622", "metadata": {}, "source": [ "# LoadTable returns a similar response\n", "However, the S3 credentials are scoped to read-only" ] }, { "cell_type": "code", "execution_count": null, "id": "10939f27-e168-4e62-8fb3-6501dad6e16a", "metadata": { "editable": true, "slideshow": { "slide_type": "" }, "tags": [] }, "outputs": [], "source": [ "tbl_meta = pm_client.load_table(prefix=catalog_name, namespace=format_namespace(['COLLADO_TEST', 'PUBLIC']), table='TEST_TABLE', x_iceberg_access_delegation='true')\n", "display(JSON(tbl_meta.to_dict(), expanded=True))" ] }, { "cell_type": "markdown", "id": "967f00a6-919b-4899-aa4f-23f9d21db55c", "metadata": {}, "source": [ "# Metadata manipulation is blocked by Polaris\n", "PMs are always dropping tables in prod" ] }, { "cell_type": "code", "execution_count": null, "id": "ee339f9b-9c10-4afe-bb4a-20e49c93e4c6", "metadata": { "editable": true, "slideshow": { "slide_type": "" }, "tags": [ "raises-exception" ] }, "outputs": [], "source": [ "pm_client.drop_table(prefix=catalog_name, namespace=format_namespace(['COLLADO_TEST', 'PUBLIC']), table='TEST_TABLE')" ] }, { "cell_type": "markdown", "id": "e2927ba4-1874-4df2-83b8-b81c05f07a75", "metadata": {}, "source": [ "# Add another Principal Role to the Engineer Principal\n", "A principal can belong to multiple Principal Roles. Typically, a call will use the union of all privilages assigned to all of the principal's roles. " ] }, { "cell_type": "code", "execution_count": null, "id": "44a1cfbd-40c4-44d1-9ffc-b1a70e7ad122", "metadata": {}, "outputs": [], "source": [ "# Create a new principal role\n", "ops_role = create_principal_role(root_client, \"ops_engineer\")\n", "\n", "# Grant the read_only catalog role to the new principal role\n", "root_client.assign_catalog_role_to_principal_role(principal_role_name=ops_role.name,\n", " catalog_name=catalog.name,\n", " grant_catalog_role_request=GrantCatalogRoleRequest(catalog_role=read_only_role))\n", "\n", "# Assign the engineer principal to the new role\n", "# The engineer principal now belongs to _both_ roles\n", "root_client.assign_principal_role(engineer_principal.principal.name, grant_principal_role_request=GrantPrincipalRoleRequest(principal_role=ops_role))" ] }, { "cell_type": "markdown", "id": "ce2afe6d-8626-46bd-a120-d6369a34c2ee", "metadata": {}, "source": [ "# Scope the spark session to a single role\n", "In this case, the Spark session is down-scoped to only the role specified. Even though the engineer has read-write privileges, the session only has privileges assigned to the specified Principal Role - in this case, the `read_only` catalog role." ] }, { "cell_type": "code", "execution_count": null, "id": "fd2b4ab0-2368-432b-8755-27933b58b297", "metadata": {}, "outputs": [], "source": [ "ro_spark = spark.newSession()\n", "ro_spark.conf.set(\"spark.sql.catalog.polaris.scope\", 'PRINCIPAL_ROLE:ops_engineer')\n", "ro_spark.sql(\"USE polaris\")\n", "ro_spark.sql(\"USE NAMESPACE COLLADO_TEST.PUBLIC\")\n", "ro_spark.sql(\"SHOW TABLES\").show()" ] }, { "cell_type": "markdown", "id": "ba2d8968-6cda-4193-9d44-0303afcdbb4a", "metadata": {}, "source": [ "# The engineer can still read data" ] }, { "cell_type": "code", "execution_count": null, "id": "3f04f339-1c36-46fe-bb48-9f479171fffd", "metadata": { "editable": true, "slideshow": { "slide_type": "" }, "tags": [] }, "outputs": [], "source": [ "ro_spark.sql(\"SELECT * FROM TEST_TABLE\").show()" ] }, { "cell_type": "markdown", "id": "242cf2f2-4a8b-4224-9352-d21d47d15010", "metadata": {}, "source": [ "# But inserts fail" ] }, { "cell_type": "code", "execution_count": null, "id": "f3166e9d-2201-43aa-88fd-64ef8f7c2dcf", "metadata": { "editable": true, "slideshow": { "slide_type": "" }, "tags": [ "raises-exception" ] }, "outputs": [], "source": [ "ro_spark.sql(\"INSERT INTO TEST_TABLE VALUES (4, 'you cannot see this data'), (5, 'it will never be inserted'), (6, 'sad emoji')\")" ] }, { "cell_type": "markdown", "id": "89966ff5-6bfe-477b-8474-7ec7d0df9aa8", "metadata": {}, "source": [ "# And metadata operations are prohibited\n", "Oops - I didn't mean to drop the _production_ table!" ] }, { "cell_type": "code", "execution_count": null, "id": "69f89060-9368-4556-bb32-cb515f905f48", "metadata": { "editable": true, "slideshow": { "slide_type": "" }, "tags": [] }, "outputs": [], "source": [ "# create a token client with the _engineer's_ credentials\n", "token_client = CatalogApiClient(CatalogApiClientConfiguration(username=engineer_principal.credentials.client_id,\n", " password=engineer_principal.credentials.client_secret,\n", " host='http://polaris:8181/api/catalog'))\n", "\n", "# specify the role I want to activate - only ops_engineer\n", "ops_token = IcebergOAuth2API(token_client).get_token(scope='PRINCIPAL_ROLE:ops_engineer',\n", " client_id=engineer_principal.credentials.client_id,\n", " client_secret=engineer_principal.credentials.client_secret,\n", " grant_type='client_credentials',\n", " _headers={'realm': 'default-realm'})\n", "\n", "# The returned token is scoped to _only_ the privileges granted to the ops_engineer role\n", "# The ops_client fails to do any real damage even though the engineer normally has DROP_TABLE privileges\n", "ops_client = IcebergCatalogAPI(CatalogApiClient(CatalogApiClientConfiguration(access_token=ops_token.access_token,\n", " host='http://polaris:8181/api/catalog')))\n", "ops_client.drop_table(prefix=catalog_name, namespace=format_namespace(['COLLADO_TEST', 'PUBLIC']), table='TEST_TABLE')" ] } ], "metadata": { "kernelspec": { "display_name": "Python 3 (ipykernel)", "language": "python", "name": "python3" }, "language_info": { "codemirror_mode": { "name": "ipython", "version": 3 }, "file_extension": ".py", "mimetype": "text/x-python", "name": "python", "nbconvert_exporter": "python", "pygments_lexer": "ipython3", "version": "3.11.6" }, "toc-autonumbering": false, "toc-showmarkdowntxt": false, "toc-showtags": false }, "nbformat": 4, "nbformat_minor": 5 }