|
| 1 | +--- |
| 2 | +# |
| 3 | +# Licensed to the Apache Software Foundation (ASF) under one |
| 4 | +# or more contributor license agreements. See the NOTICE file |
| 5 | +# distributed with this work for additional information |
| 6 | +# regarding copyright ownership. The ASF licenses this file |
| 7 | +# to you under the Apache License, Version 2.0 (the |
| 8 | +# "License"); you may not use this file except in compliance |
| 9 | +# with the License. You may obtain a copy of the License at |
| 10 | +# |
| 11 | +# http://www.apache.org/licenses/LICENSE-2.0 |
| 12 | +# |
| 13 | +# Unless required by applicable law or agreed to in writing, |
| 14 | +# software distributed under the License is distributed on an |
| 15 | +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
| 16 | +# KIND, either express or implied. See the License for the |
| 17 | +# specific language governing permissions and limitations |
| 18 | +# under the License. |
| 19 | +# |
| 20 | +Title: Polaris Spark Client |
| 21 | +type: docs |
| 22 | +weight: 650 |
| 23 | +--- |
| 24 | + |
| 25 | +Apache Polaris now provides Catalog support for Generic Tables (non-Iceberg tables), please check out |
| 26 | +the [Catalog API Spec]({{% ref "polaris-catalog-service" %}}) for Generic Table API specs. |
| 27 | + |
| 28 | +Along with the Generic Table Catalog support, Polaris is also releasing a Spark client, which helps to |
| 29 | +provide an end-to-end solution for Apache Spark to manage Delta tables using Polaris. |
| 30 | + |
| 31 | +Note the Polaris Spark client is able to handle both Iceberg and Delta tables, not just Delta. |
| 32 | + |
| 33 | +This page documents how to connect Spark with Polaris Service using the Polaris Spark client. |
| 34 | + |
| 35 | +## Quick Start with Local Polaris service |
| 36 | +If you want to quickly try out the functionality with a local Polaris service, simply check out the Polaris repo |
| 37 | +and follow the instructions in the Spark plugin getting-started |
| 38 | +[README](https://github.com/apache/polaris/blob/main/plugins/spark/v3.5/getting-started/README.md). |
| 39 | + |
| 40 | +Check out the Polaris repo: |
| 41 | +```shell |
| 42 | +cd ~ |
| 43 | +git clone https://github.com/apache/polaris.git |
| 44 | +``` |
| 45 | + |
| 46 | +## Start Spark against a deployed Polaris service |
| 47 | +Before starting, ensure that the deployed Polaris service supports Generic Tables, and that Spark 3.5(version 3.5.3 or later is installed). |
| 48 | +Spark 3.5.5 is recommended, and you can follow the instructions below to get a Spark 3.5.5 distribution. |
| 49 | +```shell |
| 50 | +cd ~ |
| 51 | +wget https://archive.apache.org/dist/spark/spark-3.5.5/spark-3.5.5-bin-hadoop3.tgz |
| 52 | +mkdir spark-3.5 |
| 53 | +tar xzvf spark-3.5.5-bin-hadoop3.tgz -C spark-3.5 --strip-components=1 |
| 54 | +cd spark-3.5 |
| 55 | +``` |
| 56 | + |
| 57 | +### Connecting with Spark using the Polaris Spark client |
| 58 | +The following CLI command can be used to start the Spark with connection to the deployed Polaris service using |
| 59 | +a released Polaris Spark client. |
| 60 | + |
| 61 | +```shell |
| 62 | +bin/spark-shell \ |
| 63 | +--packages <polaris-spark-client-package>,org.apache.hadoop:hadoop-aws:3.4.0,io.delta:delta-spark_2.12:3.3.1 \ |
| 64 | +--conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions,io.delta.sql.DeltaSparkSessionExtension \ |
| 65 | +--conf spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog \ |
| 66 | +--conf spark.sql.catalog.<spark-catalog-name>.warehouse=<polaris-catalog-name> \ |
| 67 | +--conf spark.sql.catalog.<spark-catalog-name>.header.X-Iceberg-Access-Delegation=vended-credentials \ |
| 68 | +--conf spark.sql.catalog.<spark-catalog-name>=org.apache.polaris.spark.SparkCatalog \ |
| 69 | +--conf spark.sql.catalog.<spark-catalog-name>.uri=<polaris-service-uri> \ |
| 70 | +--conf spark.sql.catalog.<spark-catalog-name>.credential='<client-id>:<client-secret>' \ |
| 71 | +--conf spark.sql.catalog.<spark-catalog-name>.scope='PRINCIPAL_ROLE:ALL' \ |
| 72 | +--conf spark.sql.catalog.<spark-catalog-name>.token-refresh-enabled=true |
| 73 | +``` |
| 74 | +Assume the released Polaris Spark client you want to use is `org.apache.polaris:polaris-iceberg-1.8.1-spark-runtime-3.5_2.12:1.0.0`, |
| 75 | +replace the `polaris-spark-client-package` field with the release. |
| 76 | + |
| 77 | +The `spark-catalog-name` is the catalog name you will use with Spark, and `polaris-catalog-name` is the catalog name used |
| 78 | +by Polaris service, for simplicity, you can use the same name. |
| 79 | + |
| 80 | +Replace the `polaris-service-uri` with the uri of the deployed Polaris service. For example, with a locally deployed |
| 81 | +Polaris service, the uri would be `http://localhost:8181/api/catalog`. |
| 82 | + |
| 83 | +For `client-id` and `client-secret` values, you can refer to [Using Polaris]({{% ref "getting-started/using-polaris" %}}) |
| 84 | +for more details. |
| 85 | + |
| 86 | +You can also start the connection by programmatically initialize a SparkSession, following is an example with PySpark: |
| 87 | +```python |
| 88 | +from pyspark.sql import SparkSession |
| 89 | + |
| 90 | +spark = SparkSession.builder |
| 91 | + .config("spark.jars.packages", "<polaris-spark-client-package>,org.apache.hadoop:hadoop-aws:3.3.4,io.delta:delta-spark_2.12:3.3.1") |
| 92 | + .config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") |
| 93 | + .config("spark.sql.extensions", "org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions,io.delta.sql.DeltaSparkSessionExtension") |
| 94 | + .config("spark.sql.catalog.<spark-catalog-name>", "org.apache.polaris.spark.SparkCatalog") |
| 95 | + .config("spark.sql.catalog.<spark-catalog-name>.uri", <polaris-service-uri>) |
| 96 | + .config("spark.sql.catalog.<spark-catalog-name>.token-refresh-enabled", "true") |
| 97 | + .config("spark.sql.catalog.<spark-catalog-name>.credential", "<client-id>:<client_secret>") |
| 98 | + .config("spark.sql.catalog.<spark-catalog-name>.warehouse", <polaris_catalog_name>) |
| 99 | + .config("spark.sql.catalog.polaris.scope", 'PRINCIPAL_ROLE:ALL') |
| 100 | + .config("spark.sql.catalog.polaris.header.X-Iceberg-Access-Delegation", 'vended-credentials') |
| 101 | + .getOrCreate() |
| 102 | +``` |
| 103 | +Similar as the CLI command, make sure the corresponding fields are replaced correctly. |
| 104 | + |
| 105 | +### Create tables with Spark |
| 106 | +After Spark is started, you can use it to create and access Iceberg and Delta tables, for example: |
| 107 | +```python |
| 108 | +spark.sql("USE polaris") |
| 109 | +spark.sql("CREATE NAMESPACE IF NOT EXISTS DELTA_NS") |
| 110 | +spark.sql("CREATE NAMESPACE IF NOT EXISTS DELTA_NS.PUBLIC") |
| 111 | +spark.sql("USE NAMESPACE DELTA_NS.PUBLIC") |
| 112 | +spark.sql("""CREATE TABLE IF NOT EXISTS PEOPLE ( |
| 113 | + id int, name string) |
| 114 | +USING delta LOCATION 'file:///tmp/var/delta_tables/people'; |
| 115 | +""") |
| 116 | +``` |
| 117 | + |
| 118 | +## Connecting with Spark using local Polaris Spark client jar |
| 119 | +If you would like to use a version of the Spark client that is currently not yet released, you can |
| 120 | +build a Spark client jar locally from source. Please check out the Polaris repo and refer to the Spark plugin |
| 121 | +[README](https://github.com/apache/polaris/blob/main/plugins/spark/README.md) for detailed instructions. |
| 122 | + |
| 123 | +## Limitations |
| 124 | +The Polaris Spark client has the following functionality limitations: |
| 125 | +1) Create table as select (CTAS) is not supported for Delta tables. As a result, the `saveAsTable` method of `Dataframe` |
| 126 | + is also not supported, since it relies on the CTAS support. |
| 127 | +2) Create a Delta table without explicit location is not supported. |
| 128 | +3) Rename a Delta table is not supported. |
| 129 | +4) ALTER TABLE ... SET LOCATION/SET FILEFORMAT/ADD PARTITION is not supported for DELTA table. |
| 130 | +5) For other non-Iceberg tables like csv, it is not supported. |
0 commit comments