Skip to content

Commit

Permalink
Add DatabricksTaskOperator (apache#40013)
Browse files Browse the repository at this point in the history
This pull request introduces the [DatabricksTaskOperator](https://github.com/astronomer/astro-provider-databricks/blob/main/src/astro_databricks/operators/common.py#L26) 
to the Airflow Databricks provider from the [astro-provider-databricks](https://github.com/astronomer/astro-provider-databricks/tree/main)
repository. Unlike the `DatabricksNotebookOperator` which only
allows to run Notebook tasks, the `DatabricksTaskOperator` allows
running all kinds of tasks across a wide range of types including 
notebooks, JAR files, Python scripts, Databricks SQL queries 
and dashboards, Delta Live Tables pipelines, dbt tasks and more
that are supported by the [Jobs API in its tasks attribute](https://docs.databricks.com/api/workspace/jobs/create#tasks).

It marks another pull request aimed at contributing  operators 
and features from that repository into the Airflow  Databricks 
provider. This PR also abstracts the common implementation
between the `DatabricksNotebookOperator` and 
`DatabricksTaskOperator` into a base abstract class 
`DatabricksTaskBaseOperator` so that we do not introduce 
duplicate code for the common set of implementation methods.
  • Loading branch information
pankajkoti authored Jun 4, 2024
1 parent 32fd29d commit 68bd42a
Show file tree
Hide file tree
Showing 6 changed files with 516 additions and 165 deletions.
460 changes: 310 additions & 150 deletions airflow/providers/databricks/operators/databricks.py

Large diffs are not rendered by default.

1 change: 1 addition & 0 deletions airflow/providers/databricks/provider.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -95,6 +95,7 @@ integrations:
- /docs/apache-airflow-providers-databricks/operators/notebook.rst
- /docs/apache-airflow-providers-databricks/operators/submit_run.rst
- /docs/apache-airflow-providers-databricks/operators/run_now.rst
- /docs/apache-airflow-providers-databricks/operators/task.rst
logo: /integration-logos/databricks/Databricks.png
tags: [service]
- integration-name: Databricks SQL
Expand Down
46 changes: 46 additions & 0 deletions docs/apache-airflow-providers-databricks/operators/task.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.
.. _howto/operator:DatabricksTaskOperator:


DatabricksTaskOperator
======================

Use the :class:`~airflow.providers.databricks.operators.databricks.DatabricksTaskOperator` to launch and monitor
task runs on Databricks as Airflow tasks. This can be used as a standalone operator in a DAG and as well as part of a
Databricks Workflow by using it as an operator(task) within the
:class:`~airflow.providers.databricks.operators.databricks_workflow.DatabricksWorkflowTaskGroup`.



Examples
--------

Running a notebook in Databricks using DatabricksTaskOperator
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
.. exampleinclude:: /../../tests/system/providers/databricks/example_databricks.py
:language: python
:start-after: [START howto_operator_databricks_task_notebook]
:end-before: [END howto_operator_databricks_task_notebook]

Running a SQL query in Databricks using DatabricksTaskOperator
~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
.. exampleinclude:: /../../tests/system/providers/databricks/example_databricks.py
:language: python
:start-after: [START howto_operator_databricks_task_sql]
:end-before: [END howto_operator_databricks_task_sql]
96 changes: 83 additions & 13 deletions tests/providers/databricks/operators/test_databricks.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@
DatabricksRunNowOperator,
DatabricksSubmitRunDeferrableOperator,
DatabricksSubmitRunOperator,
DatabricksTaskBaseOperator,
DatabricksTaskOperator,
)
from airflow.providers.databricks.triggers.databricks import DatabricksExecutionTrigger
from airflow.providers.databricks.utils import databricks as utils
Expand Down Expand Up @@ -1832,20 +1834,30 @@ def test_databricks_run_now_deferrable_operator_success_before_defer(self, mock_


class TestDatabricksNotebookOperator:
def test_is_instance_of_databricks_task_base_operator(self):
operator = DatabricksNotebookOperator(
task_id="test_task",
notebook_path="test_path",
source="test_source",
databricks_conn_id="test_conn_id",
)

assert isinstance(operator, DatabricksTaskBaseOperator)

def test_execute_with_wait_for_termination(self):
operator = DatabricksNotebookOperator(
task_id="test_task",
notebook_path="test_path",
source="test_source",
databricks_conn_id="test_conn_id",
)
operator.launch_notebook_job = MagicMock(return_value=12345)
operator._launch_job = MagicMock(return_value=12345)
operator.monitor_databricks_job = MagicMock()

operator.execute({})

assert operator.wait_for_termination is True
operator.launch_notebook_job.assert_called_once()
operator._launch_job.assert_called_once()
operator.monitor_databricks_job.assert_called_once()

def test_execute_without_wait_for_termination(self):
Expand All @@ -1856,18 +1868,24 @@ def test_execute_without_wait_for_termination(self):
databricks_conn_id="test_conn_id",
wait_for_termination=False,
)
operator.launch_notebook_job = MagicMock(return_value=12345)
operator._launch_job = MagicMock(return_value=12345)
operator.monitor_databricks_job = MagicMock()

operator.execute({})

assert operator.wait_for_termination is False
operator.launch_notebook_job.assert_called_once()
operator._launch_job.assert_called_once()
operator.monitor_databricks_job.assert_not_called()

@mock.patch("airflow.providers.databricks.operators.databricks.DatabricksHook")
def test_execute_with_deferrable(self, mock_databricks_hook):
mock_databricks_hook.return_value.get_run.return_value = {"state": {"life_cycle_state": "PENDING"}}
@mock.patch(
"airflow.providers.databricks.operators.databricks.DatabricksNotebookOperator._get_current_databricks_task"
)
def test_execute_with_deferrable(self, mock_get_current_task, mock_databricks_hook):
mock_databricks_hook.return_value.get_run.return_value = {
"state": {"life_cycle_state": "PENDING"},
"run_page_url": "test_url",
}
operator = DatabricksNotebookOperator(
task_id="test_task",
notebook_path="test_path",
Expand All @@ -1886,13 +1904,17 @@ def test_execute_with_deferrable(self, mock_databricks_hook):
assert exec_info.value.method_name == "execute_complete"

@mock.patch("airflow.providers.databricks.operators.databricks.DatabricksHook")
def test_execute_with_deferrable_early_termination(self, mock_databricks_hook):
@mock.patch(
"airflow.providers.databricks.operators.databricks.DatabricksNotebookOperator._get_current_databricks_task"
)
def test_execute_with_deferrable_early_termination(self, mock_get_current_task, mock_databricks_hook):
mock_databricks_hook.return_value.get_run.return_value = {
"state": {
"life_cycle_state": "TERMINATED",
"result_state": "FAILED",
"state_message": "FAILURE",
}
},
"run_page_url": "test_url",
}
operator = DatabricksNotebookOperator(
task_id="test_task",
Expand All @@ -1910,9 +1932,15 @@ def test_execute_with_deferrable_early_termination(self, mock_databricks_hook):
assert exception_message == str(exec_info.value)

@mock.patch("airflow.providers.databricks.operators.databricks.DatabricksHook")
def test_monitor_databricks_job_successful_raises_no_exception(self, mock_databricks_hook):
@mock.patch(
"airflow.providers.databricks.operators.databricks.DatabricksNotebookOperator._get_current_databricks_task"
)
def test_monitor_databricks_job_successful_raises_no_exception(
self, mock_get_current_task, mock_databricks_hook
):
mock_databricks_hook.return_value.get_run.return_value = {
"state": {"life_cycle_state": "TERMINATED", "result_state": "SUCCESS"}
"state": {"life_cycle_state": "TERMINATED", "result_state": "SUCCESS"},
"run_page_url": "test_url",
}

operator = DatabricksNotebookOperator(
Expand All @@ -1926,9 +1954,13 @@ def test_monitor_databricks_job_successful_raises_no_exception(self, mock_databr
operator.monitor_databricks_job()

@mock.patch("airflow.providers.databricks.operators.databricks.DatabricksHook")
def test_monitor_databricks_job_failed(self, mock_databricks_hook):
@mock.patch(
"airflow.providers.databricks.operators.databricks.DatabricksNotebookOperator._get_current_databricks_task"
)
def test_monitor_databricks_job_failed(self, mock_get_current_task, mock_databricks_hook):
mock_databricks_hook.return_value.get_run.return_value = {
"state": {"life_cycle_state": "TERMINATED", "result_state": "FAILED", "state_message": "FAILURE"}
"state": {"life_cycle_state": "TERMINATED", "result_state": "FAILED", "state_message": "FAILURE"},
"run_page_url": "test_url",
}

operator = DatabricksNotebookOperator(
Expand Down Expand Up @@ -1956,7 +1988,7 @@ def test_launch_notebook_job(self, mock_databricks_hook):
)
operator._hook.submit_run.return_value = 12345

run_id = operator.launch_notebook_job()
run_id = operator._launch_job()

assert run_id == 12345

Expand Down Expand Up @@ -2133,3 +2165,41 @@ def test_convert_to_databricks_workflow_task_cluster_conflict(self):
match="Both existing_cluster_id and job_cluster_key are set. Only one can be set per task.",
):
operator._convert_to_databricks_workflow_task(relevant_upstreams)


class TestDatabricksTaskOperator:
def test_is_instance_of_databricks_task_base_operator(self):
task_config = {
"sql_task": {
"query": {
"query_id": "c9cf6468-babe-41a6-abc3-10ac358c71ee",
},
"warehouse_id": "cf414a2206dfb397",
}
}
operator = DatabricksTaskOperator(
task_id="test_task",
databricks_conn_id="test_conn_id",
task_config=task_config,
)

assert isinstance(operator, DatabricksTaskBaseOperator)

def test_get_task_base_json(self):
task_config = {
"sql_task": {
"query": {
"query_id": "c9cf646-8babe-41a6-abc3-10ac358c71ee",
},
"warehouse_id": "cf414a2206dfb397",
}
}
operator = DatabricksTaskOperator(
task_id="test_task",
databricks_conn_id="test_conn_id",
task_config=task_config,
)
task_base_json = operator._get_task_base_json()

assert operator.task_config == task_config
assert task_base_json == task_config
37 changes: 37 additions & 0 deletions tests/system/providers/databricks/example_databricks.py
Original file line number Diff line number Diff line change
Expand Up @@ -42,11 +42,15 @@
DatabricksNotebookOperator,
DatabricksRunNowOperator,
DatabricksSubmitRunOperator,
DatabricksTaskOperator,
)

ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
DAG_ID = "example_databricks_operator"

QUERY_ID = os.environ.get("QUERY_ID", "c9cf6468-babe-41a6-abc3-10ac358c71ee")
WAREHOUSE_ID = os.environ.get("WAREHOUSE_ID", "cf414a2206dfb397")

with DAG(
dag_id=DAG_ID,
schedule="@daily",
Expand Down Expand Up @@ -201,6 +205,39 @@
)
# [END howto_operator_databricks_notebook_existing_cluster]

# [START howto_operator_databricks_task_notebook]
task_operator_nb_1 = DatabricksTaskOperator(
task_id="nb_1",
databricks_conn_id="databricks_conn",
job_cluster_key="Shared_job_cluster",
task_config={
"notebook_task": {
"notebook_path": "/Shared/Notebook_1",
"source": "WORKSPACE",
},
"libraries": [
{"pypi": {"package": "Faker"}},
{"pypi": {"package": "simplejson"}},
],
},
)
# [END howto_operator_databricks_task_notebook]

# [START howto_operator_databricks_task_sql]
task_operator_sql_query = DatabricksTaskOperator(
task_id="sql_query",
databricks_conn_id="databricks_conn",
task_config={
"sql_task": {
"query": {
"query_id": QUERY_ID,
},
"warehouse_id": WAREHOUSE_ID,
}
},
)
# [END howto_operator_databricks_task_sql]

from tests.system.utils.watcher import watcher

# This test needs watcher in order to properly mark success/failure
Expand Down
41 changes: 39 additions & 2 deletions tests/system/providers/databricks/example_databricks_workflow.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,10 @@
from datetime import timedelta

from airflow.models.dag import DAG
from airflow.providers.databricks.operators.databricks import DatabricksNotebookOperator
from airflow.providers.databricks.operators.databricks import (
DatabricksNotebookOperator,
DatabricksTaskOperator,
)
from airflow.providers.databricks.operators.databricks_workflow import DatabricksWorkflowTaskGroup
from airflow.utils.timezone import datetime

Expand All @@ -35,6 +38,9 @@
GROUP_ID = os.getenv("DATABRICKS_GROUP_ID", "1234").replace(".", "_")
USER = os.environ.get("USER")

QUERY_ID = os.environ.get("QUERY_ID", "c9cf6468-babe-41a6-abc3-10ac358c71ee")
WAREHOUSE_ID = os.environ.get("WAREHOUSE_ID", "cf414a2206dfb397")

job_cluster_spec = [
{
"job_cluster_key": "Shared_job_cluster",
Expand Down Expand Up @@ -95,6 +101,7 @@
job_cluster_key="Shared_job_cluster",
execution_timeout=timedelta(seconds=600),
)

notebook_2 = DatabricksNotebookOperator(
task_id="workflow_notebook_2",
databricks_conn_id=DATABRICKS_CONN_ID,
Expand All @@ -103,7 +110,37 @@
job_cluster_key="Shared_job_cluster",
notebook_params={"foo": "bar", "ds": "{{ ds }}"},
)
notebook_1 >> notebook_2

task_operator_nb_1 = DatabricksTaskOperator(
task_id="nb_1",
databricks_conn_id="databricks_conn",
job_cluster_key="Shared_job_cluster",
task_config={
"notebook_task": {
"notebook_path": "/Shared/Notebook_1",
"source": "WORKSPACE",
},
"libraries": [
{"pypi": {"package": "Faker"}},
{"pypi": {"package": "simplejson"}},
],
},
)

sql_query = DatabricksTaskOperator(
task_id="sql_query",
databricks_conn_id="databricks_conn",
task_config={
"sql_task": {
"query": {
"query_id": QUERY_ID,
},
"warehouse_id": WAREHOUSE_ID,
}
},
)

notebook_1 >> notebook_2 >> task_operator_nb_1 >> sql_query
# [END howto_databricks_workflow_notebook]

from tests.system.utils.watcher import watcher
Expand Down

0 comments on commit 68bd42a

Please sign in to comment.