Skip to content

Commit

Permalink
Add Azure synapse operator (apache#26038)
Browse files Browse the repository at this point in the history
This PR adds implementation for `AzureSynapseRunSparkBatchOperator` which allows Airflow users to submit Spark batch jobs on Azure Synapse Apache Spark Pools
  • Loading branch information
phanikumv authored Sep 1, 2022
1 parent 479ab54 commit 4bd0734
Show file tree
Hide file tree
Showing 9 changed files with 797 additions and 1 deletion.
194 changes: 194 additions & 0 deletions airflow/providers/microsoft/azure/hooks/synapse.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,194 @@
# 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.

import time
from typing import Any, Dict, Optional, Set, Union

from azure.identity import ClientSecretCredential, DefaultAzureCredential
from azure.synapse.spark import SparkClient
from azure.synapse.spark.models import SparkBatchJobOptions

from airflow.exceptions import AirflowTaskTimeout
from airflow.hooks.base import BaseHook

Credentials = Union[ClientSecretCredential, DefaultAzureCredential]


class AzureSynapseSparkBatchRunStatus:
"""Azure Synapse Spark Job operation statuses."""

NOT_STARTED = 'not_started'
STARTING = 'starting'
RUNNING = 'running'
IDLE = 'idle'
BUSY = 'busy'
SHUTTING_DOWN = 'shutting_down'
ERROR = 'error'
DEAD = 'dead'
KILLED = 'killed'
SUCCESS = 'success'

TERMINAL_STATUSES = {SUCCESS, DEAD, KILLED, ERROR}


class AzureSynapseHook(BaseHook):
"""
A hook to interact with Azure Synapse.
:param azure_synapse_conn_id: The :ref:`Azure Synapse connection id<howto/connection:synapse>`.
:param spark_pool: The Apache Spark pool used to submit the job
"""

conn_type: str = 'azure_synapse'
conn_name_attr: str = 'azure_synapse_conn_id'
default_conn_name: str = 'azure_synapse_default'
hook_name: str = 'Azure Synapse'

@staticmethod
def get_connection_form_widgets() -> Dict[str, Any]:
"""Returns connection widgets to add to connection form"""
from flask_appbuilder.fieldwidgets import BS3TextFieldWidget
from flask_babel import lazy_gettext
from wtforms import StringField

return {
"extra__azure_synapse__tenantId": StringField(
lazy_gettext('Tenant ID'), widget=BS3TextFieldWidget()
),
"extra__azure_synapse__subscriptionId": StringField(
lazy_gettext('Subscription ID'), widget=BS3TextFieldWidget()
),
}

@staticmethod
def get_ui_field_behaviour() -> Dict[str, Any]:
"""Returns custom field behaviour"""
return {
"hidden_fields": ['schema', 'port', 'extra'],
"relabeling": {'login': 'Client ID', 'password': 'Secret', 'host': 'Synapse Workspace URL'},
}

def __init__(self, azure_synapse_conn_id: str = default_conn_name, spark_pool: str = ''):
self.job_id: Optional[int] = None
self._conn: Optional[SparkClient] = None
self.conn_id = azure_synapse_conn_id
self.spark_pool = spark_pool
super().__init__()

def get_conn(self) -> SparkClient:
if self._conn is not None:
return self._conn

conn = self.get_connection(self.conn_id)
tenant = conn.extra_dejson.get('extra__azure_synapse__tenantId')
spark_pool = self.spark_pool
livy_api_version = "2022-02-22-preview"

try:
subscription_id = conn.extra_dejson['extra__azure_synapse__subscriptionId']
except KeyError:
raise ValueError("A Subscription ID is required to connect to Azure Synapse.")

credential: Credentials
if conn.login is not None and conn.password is not None:
if not tenant:
raise ValueError("A Tenant ID is required when authenticating with Client ID and Secret.")

credential = ClientSecretCredential(
client_id=conn.login, client_secret=conn.password, tenant_id=tenant
)
else:
credential = DefaultAzureCredential()

self._conn = self._create_client(credential, conn.host, spark_pool, livy_api_version, subscription_id)

return self._conn

@staticmethod
def _create_client(credential: Credentials, host, spark_pool, livy_api_version, subscription_id: str):
return SparkClient(
credential=credential,
endpoint=host,
spark_pool_name=spark_pool,
livy_api_version=livy_api_version,
subscription_id=subscription_id,
)

def run_spark_job(
self,
payload: SparkBatchJobOptions,
):
"""
Run a job in an Apache Spark pool.
:param payload: Livy compatible payload which represents the spark job that a user wants to submit.
"""
job = self.get_conn().spark_batch.create_spark_batch_job(payload)
self.job_id = job.id
return job

def get_job_run_status(self):
"""Get the job run status."""
job_run_status = self.get_conn().spark_batch.get_spark_batch_job(batch_id=self.job_id).state
return job_run_status

def wait_for_job_run_status(
self,
job_id: Optional[int],
expected_statuses: Union[str, Set[str]],
check_interval: int = 60,
timeout: int = 60 * 60 * 24 * 7,
) -> bool:
"""
Waits for a job run to match an expected status.
:param job_id: The job run identifier.
:param expected_statuses: The desired status(es) to check against a job run's current status.
:param check_interval: Time in seconds to check on a job run's status.
:param timeout: Time in seconds to wait for a job to reach a terminal status or the expected
status.
"""
job_run_status = self.get_job_run_status()
start_time = time.monotonic()

while (
job_run_status not in AzureSynapseSparkBatchRunStatus.TERMINAL_STATUSES
and job_run_status not in expected_statuses
):
# Check if the job-run duration has exceeded the ``timeout`` configured.
if start_time + timeout < time.monotonic():
raise AirflowTaskTimeout(
f"Job {job_id} has not reached a terminal status after {timeout} seconds."
)

# Wait to check the status of the job run based on the ``check_interval`` configured.
self.log.info("Sleeping for %s seconds", str(check_interval))
time.sleep(check_interval)

job_run_status = self.get_job_run_status()
self.log.info("Current spark job run status is %s", job_run_status)

return job_run_status in expected_statuses

def cancel_job_run(
self,
job_id: int,
) -> None:
"""
Cancel the spark job run.
:param job_id: The synapse spark job identifier.
"""
self.get_conn().spark_batch.cancel_spark_batch_job(job_id)
106 changes: 106 additions & 0 deletions airflow/providers/microsoft/azure/operators/synapse.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,106 @@
# 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 typing import TYPE_CHECKING, Sequence

from azure.synapse.spark.models import SparkBatchJobOptions

from airflow.models import BaseOperator
from airflow.providers.microsoft.azure.hooks.synapse import AzureSynapseHook, AzureSynapseSparkBatchRunStatus

if TYPE_CHECKING:
from airflow.utils.context import Context


class AzureSynapseRunSparkBatchOperator(BaseOperator):
"""
Executes a Spark job on Azure Synapse.
.. see also::
For more information on how to use this operator, take a look at the guide:
:ref:`howto/operator:AzureSynapseRunSparkBatchOperator`
:param azure_synapse_conn_id: The connection identifier for connecting to Azure Synapse.
:param wait_for_termination: Flag to wait on a job run's termination.
:param spark_pool: The target synapse spark pool used to submit the job
:param payload: Livy compatible payload which represents the spark job that a user wants to submit
:param timeout: Time in seconds to wait for a job to reach a terminal status for non-asynchronous
waits. Used only if ``wait_for_termination`` is True.
:param check_interval: Time in seconds to check on a job run's status for non-asynchronous waits.
Used only if ``wait_for_termination`` is True.
"""

template_fields: Sequence[str] = (
"azure_synapse_conn_id",
"spark_pool",
)
template_fields_renderers = {"parameters": "json"}

ui_color = "#0678d4"

def __init__(
self,
*,
azure_synapse_conn_id: str = AzureSynapseHook.default_conn_name,
wait_for_termination: bool = True,
spark_pool: str = '',
payload: SparkBatchJobOptions,
timeout: int = 60 * 60 * 24 * 7,
check_interval: int = 60,
**kwargs,
) -> None:
super().__init__(**kwargs)
self.job_id = None
self.azure_synapse_conn_id = azure_synapse_conn_id
self.wait_for_termination = wait_for_termination
self.spark_pool = spark_pool
self.payload = payload
self.timeout = timeout
self.check_interval = check_interval

def execute(self, context: "Context") -> None:
self.hook = AzureSynapseHook(
azure_synapse_conn_id=self.azure_synapse_conn_id, spark_pool=self.spark_pool
)
self.log.info("Executing the Synapse spark job.")
response = self.hook.run_spark_job(payload=self.payload)
self.log.info(response)
self.job_id = vars(response)["id"]
# Push the ``job_id`` value to XCom regardless of what happens during execution. This allows for
# retrieval the executed job's ``id`` for downstream tasks especially if performing an
# asynchronous wait.
context["ti"].xcom_push(key="job_id", value=self.job_id)

if self.wait_for_termination:
self.log.info("Waiting for job run %s to terminate.", self.job_id)

if self.hook.wait_for_job_run_status(
job_id=self.job_id,
expected_statuses=AzureSynapseSparkBatchRunStatus.SUCCESS,
check_interval=self.check_interval,
timeout=self.timeout,
):
self.log.info("Job run %s has completed successfully.", self.job_id)
else:
raise Exception(f"Job run {self.job_id} has failed or has been cancelled.")

def on_kill(self) -> None:
if self.job_id:
self.hook.cancel_job_run(
job_id=self.job_id,
)
self.log.info("Job run %s has been cancelled successfully.", self.job_id)
14 changes: 14 additions & 0 deletions airflow/providers/microsoft/azure/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@ dependencies:
- azure-storage-file>=2.1.0
# Limited due to https://github.com/Azure/azure-uamqp-python/issues/191
- azure-servicebus>=7.6.1; platform_machine != "aarch64"
- azure-synapse-spark

integrations:
- integration-name: Microsoft Azure Batch
Expand Down Expand Up @@ -115,6 +116,11 @@ integrations:
how-to-guide:
- /docs/apache-airflow-providers-microsoft-azure/operators/asb.rst
tags: [azure]
- integration-name: Microsoft Azure Synapse
external-doc-url: https://azure.microsoft.com/en-us/services/synapse-analytics/
how-to-guide:
- /docs/apache-airflow-providers-microsoft-azure/operators/azure_synapse.rst
tags: [azure]

operators:
- integration-name: Microsoft Azure Data Lake Storage
Expand Down Expand Up @@ -146,6 +152,9 @@ operators:
- integration-name: Microsoft Azure Service Bus
python-modules:
- airflow.providers.microsoft.azure.operators.asb
- integration-name: Microsoft Azure Synapse
python-modules:
- airflow.providers.microsoft.azure.operators.synapse

sensors:
- integration-name: Microsoft Azure Cosmos DB
Expand Down Expand Up @@ -200,6 +209,9 @@ hooks:
- integration-name: Microsoft Azure Service Bus
python-modules:
- airflow.providers.microsoft.azure.hooks.asb
- integration-name: Microsoft Azure Synapse
python-modules:
- airflow.providers.microsoft.azure.hooks.synapse

transfers:
- source-integration-name: Local
Expand Down Expand Up @@ -252,6 +264,8 @@ connection-types:
connection-type: azure_container_registry
- hook-class-name: airflow.providers.microsoft.azure.hooks.asb.BaseAzureServiceBusHook
connection-type: azure_service_bus
- hook-class-name: airflow.providers.microsoft.azure.hooks.synapse.AzureSynapseHook
connection-type: azure_synapse

secrets-backends:
- airflow.providers.microsoft.azure.secrets.key_vault.AzureKeyVaultBackend
Expand Down
Loading

0 comments on commit 4bd0734

Please sign in to comment.