Skip to content

Commit

Permalink
Add support for Spark Connect to pyspark decorator (#35665)
Browse files Browse the repository at this point in the history
* Add support for Spark Connect to pyspark decorator

In Apache Spark 3.4 Spark Connect was introduced which
allows remote connectivity to remote Spark Cluster using
the DataFrame API.
  • Loading branch information
bolkedebruin committed Nov 16, 2023
1 parent e29464b commit 9103ea1
Show file tree
Hide file tree
Showing 8 changed files with 339 additions and 15 deletions.
27 changes: 19 additions & 8 deletions airflow/providers/apache/spark/decorators/pyspark.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory
from airflow.hooks.base import BaseHook
from airflow.operators.python import PythonOperator
from airflow.providers.apache.spark.hooks.spark_connect import SparkConnectHook

if TYPE_CHECKING:
from airflow.utils.context import Context
Expand Down Expand Up @@ -73,34 +74,44 @@ def execute(self, context: Context):
from pyspark import SparkConf
from pyspark.sql import SparkSession

conf = SparkConf().setAppName(f"{self.dag_id}-{self.task_id}")
conf = SparkConf()
conf.set("spark.app.name", f"{self.dag_id}-{self.task_id}")

master = "local[*]"
url = "local[*]"
if self.conn_id:
# we handle both spark connect and spark standalone
conn = BaseHook.get_connection(self.conn_id)
if conn.port:
master = f"{conn.host}:{conn.port}"
if conn.conn_type == SparkConnectHook.conn_type:
url = SparkConnectHook(self.conn_id).get_connection_url()
elif conn.port:
url = f"{conn.host}:{conn.port}"
elif conn.host:
master = conn.host
url = conn.host

for key, value in conn.extra_dejson.items():
conf.set(key, value)

conf.setMaster(master)
# you cannot have both remote and master
if url.startswith("sc://"):
conf.set("spark.remote", url)

# task can override connection config
for key, value in self.config_kwargs.items():
conf.set(key, value)

if not conf.get("spark.remote") and not conf.get("spark.master"):
conf.set("spark.master", url)

spark = SparkSession.builder.config(conf=conf).getOrCreate()
sc = spark.sparkContext

if not self.op_kwargs:
self.op_kwargs = {}

op_kwargs: dict[str, Any] = dict(self.op_kwargs)
op_kwargs["spark"] = spark
op_kwargs["sc"] = sc

# spark context is not available when using spark connect
op_kwargs["sc"] = spark.sparkContext if not conf.get("spark.remote") else None

self.op_kwargs = op_kwargs
return super().execute(context)
Expand Down
99 changes: 99 additions & 0 deletions airflow/providers/apache/spark/hooks/spark_connect.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
#
# 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.
from __future__ import annotations

from typing import Any
from urllib.parse import quote, urlparse, urlunparse

from airflow.hooks.base import BaseHook
from airflow.utils.log.logging_mixin import LoggingMixin


class SparkConnectHook(BaseHook, LoggingMixin):
"""Hook for Spark Connect."""

# from pyspark's ChannelBuilder
PARAM_USE_SSL = "use_ssl"
PARAM_TOKEN = "token"
PARAM_USER_ID = "user_id"

conn_name_attr = "conn_id"
default_conn_name = "spark_connect_default"
conn_type = "spark_connect"
hook_name = "Spark Connect"

@staticmethod
def get_ui_field_behaviour() -> dict[str, Any]:
"""Return custom field behaviour."""
return {
"hidden_fields": [
"schema",
],
"relabeling": {"password": "Token", "login": "User ID"},
}

@staticmethod
def get_connection_form_widgets() -> dict[str, Any]:
"""Returns connection widgets to add to connection form."""
from flask_babel import lazy_gettext
from wtforms import BooleanField

return {
SparkConnectHook.PARAM_USE_SSL: BooleanField(lazy_gettext("Use SSL"), default=False),
}

def __init__(self, conn_id: str = default_conn_name) -> None:
super().__init__()
self._conn_id = conn_id

def get_connection_url(self) -> str:
conn = self.get_connection(self._conn_id)

host = conn.host
if conn.host.find("://") == -1:
host = f"sc://{conn.host}"
if conn.port:
host = f"{conn.host}:{conn.port}"

url = urlparse(host)

if url.path:
raise ValueError("Path {url.path} is not supported in Spark Connect connection URL")

params = []

if conn.login:
params.append(f"{SparkConnectHook.PARAM_USER_ID}={quote(conn.login)}")

if conn.password:
params.append(f"{SparkConnectHook.PARAM_TOKEN}={quote(conn.password)}")

use_ssl = conn.extra_dejson.get(SparkConnectHook.PARAM_USE_SSL)
if use_ssl is not None:
params.append(f"{SparkConnectHook.PARAM_USE_SSL}={quote(str(use_ssl))}")

return urlunparse(
(
"sc",
url.netloc,
"/",
";".join(params), # params
"",
url.fragment,
)
)
4 changes: 4 additions & 0 deletions airflow/providers/apache/spark/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ versions:
dependencies:
- apache-airflow>=2.5.0
- pyspark
- grpcio-status

integrations:
- integration-name: Apache Spark
Expand All @@ -70,13 +71,16 @@ operators:
hooks:
- integration-name: Apache Spark
python-modules:
- airflow.providers.apache.spark.hooks.spark_connect
- airflow.providers.apache.spark.hooks.spark_jdbc
- airflow.providers.apache.spark.hooks.spark_jdbc_script
- airflow.providers.apache.spark.hooks.spark_sql
- airflow.providers.apache.spark.hooks.spark_submit


connection-types:
- hook-class-name: airflow.providers.apache.spark.hooks.spark_connect.SparkConnectHook
connection-type: spark_connect
- hook-class-name: airflow.providers.apache.spark.hooks.spark_jdbc.SparkJDBCHook
connection-type: spark_jdbc
- hook-class-name: airflow.providers.apache.spark.hooks.spark_sql.SparkSqlHook
Expand Down
11 changes: 10 additions & 1 deletion docs/apache-airflow-providers-apache-spark/connections/spark.rst
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ The Apache Spark connection type enables connection to Apache Spark.
Default Connection IDs
----------------------

Spark Submit and Spark JDBC hooks and operators use ``spark_default`` by default. Spark SQL hooks and operators point to ``spark_sql_default`` by default.
Spark Submit and Spark JDBC hooks and operators use ``spark_default`` by default. Spark SQL hooks and operators point to ``spark_sql_default`` by default. The Spark Connect hook uses ``spark_connect_default`` by default.

Configuring the Connection
--------------------------
Expand All @@ -45,6 +45,15 @@ Extra (optional)
* ``spark-binary`` - The command to use for Spark submit. Some distros may use ``spark2-submit``. Default ``spark-submit``. Only ``spark-submit``, ``spark2-submit`` or ``spark3-submit`` are allowed as value.
* ``namespace`` - Kubernetes namespace (``spark.kubernetes.namespace``) to divide cluster resources between multiple users (via resource quota).

User ID (optional, only applies to Spark Connect)
The user ID to authenticate with the proxy.

Token (optional, only applies to Spark Connect)
The token to authenticate with the proxy.

Use SSL (optional, only applies to Spark Connect)
Whether to use SSL when connecting.

When specifying the connection in environment variable you should specify
it using URI syntax.

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ PySpark Decorator
=================

Python callable wrapped within the ``@task.pyspark`` decorator
is injected with a SparkContext object.
is injected with a SparkSession and SparkContext object if available.

Parameters
----------
Expand All @@ -49,3 +49,24 @@ that the ``spark`` and ``sc`` objects are injected into the function.
:dedent: 4
:start-after: [START task_pyspark]
:end-before: [END task_pyspark]


Spark Connect
-------------

In `Apache Spark 3.4 <https://spark.apache.org/docs/latest/spark-connect-overview.html>`_,
Spark Connect introduced a decoupled client-server architecture
that allows remote connectivity to Spark clusters using the DataFrame API. Using
Spark Connect is the preferred way in Airflow to make use of the PySpark decorator,
because it does not require to run the Spark driver on the same host as Airflow.
To make use of Spark Connect, you prepend your host url with ``sc://``. For example,
``sc://spark-cluster:15002``.


Authentication
^^^^^^^^^^^^^^

Spark Connect does not have built-in authentication. The gRPC HTTP/2 interface however
allows the use of authentication to communicate with the Spark Connect server through
authenticating proxies. To make use of authentication make sure to create a ``Spark Connect``
connection and set the right credentials.
1 change: 1 addition & 0 deletions generated/provider_dependencies.json
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,7 @@
"apache.spark": {
"deps": [
"apache-airflow>=2.5.0",
"grpcio-status",
"pyspark"
],
"cross-providers-deps": [
Expand Down
Loading

0 comments on commit 9103ea1

Please sign in to comment.