Skip to content

Commit

Permalink
Add guide for Apache Spark operators (#8305)
Browse files Browse the repository at this point in the history
  • Loading branch information
Rafael Bottega committed May 5, 2020
1 parent c717d12 commit 487b5cc
Show file tree
Hide file tree
Showing 8 changed files with 318 additions and 1 deletion.
17 changes: 17 additions & 0 deletions airflow/providers/apache/spark/example_dags/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
#
# 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.
78 changes: 78 additions & 0 deletions airflow/providers/apache/spark/example_dags/example_spark_dag.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
#
# 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.

"""
Example Airflow DAG to submit Apache Spark applications using
`SparkSubmitOperator`, `SparkJDBCOperator` and `SparkSqlOperator`.
"""
from airflow.models import DAG
from airflow.providers.apache.spark.operators.spark_jdbc import SparkJDBCOperator
from airflow.providers.apache.spark.operators.spark_sql import SparkSqlOperator
from airflow.providers.apache.spark.operators.spark_submit import SparkSubmitOperator
from airflow.utils.dates import days_ago

args = {
'owner': 'Airflow',
'start_date': days_ago(2)
}

with DAG(
dag_id='example_spark_operator',
default_args=args,
schedule_interval=None,
tags=['example']
) as dag:
# [START howto_operator_spark_submit]
submit_job = SparkSubmitOperator(
application="${SPARK_HOME}/examples/src/main/python/pi.py",
task_id="submit_job"
)
# [END howto_operator_spark_submit]

# [START howto_operator_spark_jdbc]
jdbc_to_spark_job = SparkJDBCOperator(
cmd_type='jdbc_to_spark',
jdbc_table="foo",
spark_conf={},
spark_jars="${SPARK_HOME}/jars/postgresql-42.2.12.jar",
jdbc_driver="org.postgresql.Driver",
metastore_table="bar",
save_mode="overwrite",
save_format="JSON",
task_id="jdbc_to_spark_job"
)

spark_to_jdbc_job = SparkJDBCOperator(
cmd_type='spark_to_jdbc',
jdbc_table="foo",
spark_conf={},
spark_jars="${SPARK_HOME}/jars/postgresql-42.2.12.jar",
jdbc_driver="org.postgresql.Driver",
metastore_table="bar",
save_mode="append",
task_id="spark_to_jdbc_job"
)
# [END howto_operator_spark_jdbc]

# [START howto_operator_spark_sql]
sql_job = SparkSqlOperator(
sql="SELECT * FROM bar",
master="local",
task_id="sql_job"
)
# [END howto_operator_spark_sql]
46 changes: 46 additions & 0 deletions docs/howto/connection/jdbc.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
.. 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.
JDBC connection
===============

The JDBC connection type enables connection to a JDBC data source.

Configuring the Connection
--------------------------

Host (required)
The host to connect to.

Schema (required)
Specify the database name to be used in.

Login (required)
Specify the user name to connect to.

Password (required)
Specify the password to connect to.

Port (optional)
Port of host to connect to. Not user in ``JdbcOperator``.

Extra (optional)
Specify the extra parameters (as json dictionary) that can be used in JDBC connection. The following parameters out of the standard python parameters are supported:

* ``conn_prefix`` - Used to build the connection url in ``SparkJDBCOperator``, added in front of host (``conn_prefix`` ``host`` [: ``port`` ] / ``schema``)
* ``extra__jdbc__drv_clsname`` - Full qualified Java class name of the JDBC driver. For ``JdbcOperator``.
* ``extra__jdbc__drv_path`` - Jar filename or sequence of filenames for the JDBC driver libs. For ``JdbcOperator``.
43 changes: 43 additions & 0 deletions docs/howto/connection/spark.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
.. Licensed to the Apache Software Foundation (ASF) under one
or more contributor license agreements. See the NOTICE file
distributed with this work for additional information
regarding copyright ownership. The ASF licenses this file
to you under the Apache License, Version 2.0 (the
"License"); you may not use this file except in compliance
with the License. You may obtain a copy of the License at
.. http://www.apache.org/licenses/LICENSE-2.0
.. Unless required by applicable law or agreed to in writing,
software distributed under the License is distributed on an
"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
KIND, either express or implied. See the License for the
specific language governing permissions and limitations
under the License.
Apache Spark Connection
=======================

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, but don't use it.

Configuring the Connection
--------------------------
Host (required)
The host to connect to, it can be ``local``, ``yarn`` or an URL.

Port (optional)
Specify the port in case of host be an URL.

Extra (optional)
Specify the extra parameters (as json dictionary) that can be used in spark connection. The following parameters out of the standard python parameters are supported:

* ``queue`` - The name of the YARN queue to which the application is submitted.
* ``deploy-mode`` - Whether to deploy your driver on the worker nodes (cluster) or locally as an external client (client).
* ``spark-home`` - If passed then build the ``spark-binary`` executable path using it (``spark-home``/bin/``spark-binary``); otherwise assume that ``spark-binary`` is present in the PATH of the executing user.
* ``spark-binary`` - The command to use for Spark submit. Some distros may use ``spark2-submit``. Default ``spark-submit``.
* ``namespace`` - Kubernetes namespace (``spark.kubernetes.namespace``) to divide cluster resources between multiple users (via resource quota).
28 changes: 28 additions & 0 deletions docs/howto/operator/apache/index.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,28 @@
.. 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.
Apache Operators
================


.. toctree::
:maxdepth: 1
:glob:

*
104 changes: 104 additions & 0 deletions docs/howto/operator/apache/spark.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
.. 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.
Apache Spark Operators
======================

.. contents::
:depth: 1
:local:

Prerequisite
------------

To use ``SparkJDBCOperator`` and ``SparkSubmitOperator``, you must configure a :doc:`Spark Connection <../../connection/spark>`. For ``SparkJDBCOperator``, you must also configure a :doc:`JDBC connection <../../connection/jdbc>`.

``SparkSqlOperator`` gets all the configurations from operator parameters.

.. _howto/operator:SparkJDBCOperator:

SparkJDBCOperator
-----------------

Launches applications on a Apache Spark server, it uses ``SparkSubmitOperator`` to perform data transfers to/from JDBC-based databases.

For parameter definition take a look at :class:`~airflow.providers.apache.spark.operators.spark_jdbc.SparkJDBCOperator`.

Using the operator
""""""""""""""""""

Using ``cmd_type`` parameter, is possible to transfer data from Spark to a database (``spark_to_jdbc``) or from a database to Spark (``jdbc_to_spark``), which will write the table using the Spark command ``saveAsTable``.

.. exampleinclude:: ../../../../airflow/providers/apache/spark/example_dags/example_spark_dag.py
:language: python
:dedent: 4
:start-after: [START howto_operator_spark_jdbc]
:end-before: [END howto_operator_spark_jdbc]


Reference
"""""""""

For further information, look at `Apache Spark DataFrameWriter documentation <https://spark.apache.org/docs/2.4.5/api/scala/index.html#org.apache.spark.sql.DataFrameWriter>`_.

.. _howto/operator:SparkSqlOperator:

SparkSqlOperator
----------------

Launches applications on a Apache Spark server, it requires that the ``spark-sql`` script is in the PATH.
The operator will run the SQL query on Spark Hive metastore service, the ``sql`` parameter can be templated and be a ``.sql`` or ``.hql`` file.

For parameter definition take a look at :class:`~airflow.providers.apache.spark.operators.spark_sql.SparkSqlOperator`.

Using the operator
""""""""""""""""""

.. exampleinclude:: ../../../../airflow/providers/apache/spark/example_dags/example_spark_dag.py
:language: python
:dedent: 4
:start-after: [START howto_operator_spark_sql]
:end-before: [END howto_operator_spark_sql]

Reference
"""""""""

For further information, look at `Running the Spark SQL CLI <https://spark.apache.org/docs/latest/sql-distributed-sql-engine.html#running-the-spark-sql-cli>`_.

.. _howto/operator:SparkSubmitOperator:

SparkSubmitOperator
-------------------

Launches applications on a Apache Spark server, it uses the ``spark-submit`` script that takes care of setting up the classpath with Spark and its dependencies, and can support different cluster managers and deploy modes that Spark supports.

For parameter definition take a look at :class:`~airflow.providers.apache.spark.operators.spark_submit.SparkSubmitOperator`.

Using the operator
""""""""""""""""""

.. exampleinclude:: ../../../../airflow/providers/apache/spark/example_dags/example_spark_dag.py
:language: python
:dedent: 4
:start-after: [START howto_operator_spark_submit]
:end-before: [END howto_operator_spark_submit]

Reference
"""""""""

For further information, look at `Apache Spark submitting applications <https://spark.apache.org/docs/latest/submitting-applications.html>`_.
1 change: 1 addition & 0 deletions docs/howto/operator/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ determine what actually executes when your DAG runs.

bash
amazon/aws/index
apache/index
dingding
gcp/index
http/index
Expand Down
2 changes: 1 addition & 1 deletion docs/operators-and-hooks-ref.rst
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,7 @@ Foundation.
-

* - `Apache Spark <https://spark.apache.org/>`__
-
- :doc:`How to use <howto/operator/apache/spark>`
- :mod:`airflow.providers.apache.spark.hooks.spark_jdbc`,
:mod:`airflow.providers.apache.spark.hooks.spark_jdbc_script`,
:mod:`airflow.providers.apache.spark.hooks.spark_sql`,
Expand Down

0 comments on commit 487b5cc

Please sign in to comment.