-
Notifications
You must be signed in to change notification settings - Fork 16.4k
Add DatabricksJobsCreateOperator
#32221
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
70e1126
e39d23a
cb4252f
f113304
15635fb
446a58b
3dfa54f
020e2d7
5bec5fb
3a6ee47
7a93f00
3bf27bc
2c14c8a
914c72b
3a36a19
8d958d2
afcb498
97eb8c5
0bc9a38
c42eec9
36c5be8
1a1d2ae
022ae6f
1423ed0
13d2afe
2e3c9e3
30e2920
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -162,6 +162,135 @@ def get_link( | |
| return XCom.get_value(key=XCOM_RUN_PAGE_URL_KEY, ti_key=ti_key) | ||
|
|
||
|
|
||
| class DatabricksCreateJobsOperator(BaseOperator): | ||
| """Creates (or resets) a Databricks job using the API endpoint. | ||
|
|
||
| .. seealso:: | ||
| https://docs.databricks.com/api/workspace/jobs/create | ||
| https://docs.databricks.com/api/workspace/jobs/reset | ||
|
|
||
| :param json: A JSON object containing API parameters which will be passed | ||
| directly to the ``api/2.1/jobs/create`` endpoint. The other named parameters | ||
| (i.e. ``name``, ``tags``, ``tasks``, etc.) to this operator will | ||
| be merged with this json dictionary if they are provided. | ||
| If there are conflicts during the merge, the named parameters will | ||
| take precedence and override the top level json keys. (templated) | ||
|
|
||
| .. seealso:: | ||
| For more information about templating see :ref:`concepts:jinja-templating`. | ||
| :param name: An optional name for the job. | ||
| :param tags: A map of tags associated with the job. | ||
| :param tasks: A list of task specifications to be executed by this job. | ||
|
||
| Array of objects (JobTaskSettings). | ||
| :param job_clusters: A list of job cluster specifications that can be shared and reused by | ||
| tasks of this job. Array of objects (JobCluster). | ||
| :param email_notifications: Object (JobEmailNotifications). | ||
| :param webhook_notifications: Object (WebhookNotifications). | ||
| :param timeout_seconds: An optional timeout applied to each run of this job. | ||
| :param schedule: Object (CronSchedule). | ||
| :param max_concurrent_runs: An optional maximum allowed number of concurrent runs of the job. | ||
| :param git_source: An optional specification for a remote repository containing the notebooks | ||
| used by this job's notebook tasks. Object (GitSource). | ||
| :param access_control_list: List of permissions to set on the job. Array of object | ||
| (AccessControlRequestForUser) or object (AccessControlRequestForGroup) or object | ||
| (AccessControlRequestForServicePrincipal). | ||
|
|
||
| .. seealso:: | ||
| This will only be used on create. In order to reset ACL consider using the Databricks | ||
| UI. | ||
| :param databricks_conn_id: Reference to the | ||
| :ref:`Databricks connection <howto/connection:databricks>`. (templated) | ||
| :param polling_period_seconds: Controls the rate which we poll for the result of | ||
| this run. By default the operator will poll every 30 seconds. | ||
| :param databricks_retry_limit: Amount of times retry if the Databricks backend is | ||
| unreachable. Its value must be greater than or equal to 1. | ||
| :param databricks_retry_delay: Number of seconds to wait between retries (it | ||
| might be a floating point number). | ||
| :param databricks_retry_args: An optional dictionary with arguments passed to ``tenacity.Retrying`` class. | ||
|
|
||
| """ | ||
|
|
||
| # Used in airflow.models.BaseOperator | ||
| template_fields: Sequence[str] = ("json", "databricks_conn_id") | ||
| # Databricks brand color (blue) under white text | ||
| ui_color = "#1CB1C2" | ||
| ui_fgcolor = "#fff" | ||
|
|
||
| def __init__( | ||
| self, | ||
| *, | ||
| json: Any | None = None, | ||
|
||
| name: str | None = None, | ||
| tags: dict[str, str] | None = None, | ||
| tasks: list[dict] | None = None, | ||
| job_clusters: list[dict] | None = None, | ||
| email_notifications: dict | None = None, | ||
| webhook_notifications: dict | None = None, | ||
| timeout_seconds: int | None = None, | ||
| schedule: dict | None = None, | ||
| max_concurrent_runs: int | None = None, | ||
| git_source: dict | None = None, | ||
| access_control_list: list[dict] | None = None, | ||
| databricks_conn_id: str = "databricks_default", | ||
| polling_period_seconds: int = 30, | ||
| databricks_retry_limit: int = 3, | ||
| databricks_retry_delay: int = 1, | ||
| databricks_retry_args: dict[Any, Any] | None = None, | ||
| **kwargs, | ||
| ) -> None: | ||
| """Creates a new ``DatabricksCreateJobsOperator``.""" | ||
| super().__init__(**kwargs) | ||
| self.json = json or {} | ||
| self.databricks_conn_id = databricks_conn_id | ||
| self.polling_period_seconds = polling_period_seconds | ||
| self.databricks_retry_limit = databricks_retry_limit | ||
| self.databricks_retry_delay = databricks_retry_delay | ||
| self.databricks_retry_args = databricks_retry_args | ||
| if name is not None: | ||
stikkireddy marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| self.json["name"] = name | ||
| if tags is not None: | ||
| self.json["tags"] = tags | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Since It would be best to move modifying
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's behaviour isn't different from other databricks operators: https://github.com/apache/airflow/blob/main/airflow/providers/databricks/operators/databricks.py#L334
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. To illustrate the point, let's use this example DAG where we define the from __future__ import annotations
from pendulum import datetime
from typing import TYPE_CHECKING, Sequence
from airflow.decorators import dag, task
from airflow.models.baseoperator import BaseOperator
if TYPE_CHECKING:
from airflow.utils.context import Context
class DatabricksCreateJobsOperator(BaseOperator):
template_fields: Sequence[str] = ("json", "databricks_conn_id")
def __init__(
self,
*,
json: dict | None = None,
name: str | None = None,
tags: dict[str, str] | None = None,
**kwargs
):
super().__init__(**kwargs)
self.json = json or {}
if name is not None:
self.json["name"] = name
if tags is not None:
self.json["tags"] = tags
def execute(context: Context) -> None:
pass
@dag(start_date=datetime(2023, 1, 1), schedule=None)
def derived_template_fields():
@task
def push_json() -> dict[str, str]:
return {"key1": "val1", "key2": "val2"}
json = push_json()
DatabricksCreateJobsOperator(
task_id="create_job_w_json", json=json, name="some_name", tags={"key3": "value3"}
)
derived_template_fields()DAG parsing fails with: Even if we change the Perhaps it's possible users haven't needed a use case for predefining a Seems like there is now some movement on addressing #29069 to prevent this in the future too.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @josh-fell well spotted, that's a relevant concern! Since there is a separate ticket to address this issue, would it be okay if we consider this change outside of the scope of the current PR?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Totally fair. All of the Databricks operators would need to be cleaned up, but I also haven't heard of anyone running into this issue within their use cases. I can log a separate issue and maybe we can get some folks to contribute! It's a solid "good first issue".
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That's great, thanks a lot, @josh-fell! Please share the ticket once you create it - it's valuable and relevant.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sorry for the mighty delay here, just got around to logging the issue. |
||
| if tasks is not None: | ||
| self.json["tasks"] = tasks | ||
| if job_clusters is not None: | ||
| self.json["job_clusters"] = job_clusters | ||
| if email_notifications is not None: | ||
| self.json["email_notifications"] = email_notifications | ||
| if webhook_notifications is not None: | ||
| self.json["webhook_notifications"] = webhook_notifications | ||
| if timeout_seconds is not None: | ||
| self.json["timeout_seconds"] = timeout_seconds | ||
| if schedule is not None: | ||
| self.json["schedule"] = schedule | ||
| if max_concurrent_runs is not None: | ||
| self.json["max_concurrent_runs"] = max_concurrent_runs | ||
| if git_source is not None: | ||
| self.json["git_source"] = git_source | ||
| if access_control_list is not None: | ||
| self.json["access_control_list"] = access_control_list | ||
|
|
||
| self.json = normalise_json_content(self.json) | ||
|
|
||
| @cached_property | ||
| def _hook(self): | ||
| return DatabricksHook( | ||
| self.databricks_conn_id, | ||
| retry_limit=self.databricks_retry_limit, | ||
| retry_delay=self.databricks_retry_delay, | ||
| retry_args=self.databricks_retry_args, | ||
| caller="DatabricksCreateJobsOperator", | ||
| ) | ||
|
|
||
| def execute(self, context: Context) -> int: | ||
| if "name" not in self.json: | ||
| raise AirflowException("Missing required parameter: name") | ||
| job_id = self._hook.find_job_id_by_name(self.json["name"]) | ||
| if job_id is None: | ||
stikkireddy marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
| return self._hook.create_job(self.json) | ||
| self._hook.reset_job(str(job_id), self.json) | ||
| return job_id | ||
|
|
||
|
|
||
| class DatabricksSubmitRunOperator(BaseOperator): | ||
| """ | ||
| Submits a Spark job run to Databricks using the api/2.1/jobs/runs/submit API endpoint. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -60,11 +60,13 @@ dependencies: | |
| # The 2.9.1 (to be released soon) already contains the fix | ||
| - databricks-sql-connector>=2.0.0, <3.0.0, !=2.9.0 | ||
| - aiohttp>=3.6.3, <4 | ||
| - databricks-sdk==0.10.0 | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why limit to specific version? we rarely do this I also don't understand if this is dependency or extra dependency ? |
||
|
|
||
| integrations: | ||
| - integration-name: Databricks | ||
| external-doc-url: https://databricks.com/ | ||
| how-to-guide: | ||
| - /docs/apache-airflow-providers-databricks/operators/jobs_create.rst | ||
| - /docs/apache-airflow-providers-databricks/operators/submit_run.rst | ||
| - /docs/apache-airflow-providers-databricks/operators/run_now.rst | ||
| logo: /integration-logos/databricks/Databricks.png | ||
|
|
@@ -122,3 +124,10 @@ connection-types: | |
|
|
||
| extra-links: | ||
| - airflow.providers.databricks.operators.databricks.DatabricksJobRunLink | ||
|
|
||
| additional-extras: | ||
| # pip install apache-airflow-providers-databricks[sdk] | ||
| - name: sdk | ||
| description: Install Databricks SDK | ||
| dependencies: | ||
| - databricks-sdk==0.10.0 | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. same question |
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,91 @@ | ||
| .. 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. | ||
|
|
||
|
|
||
|
|
||
| DatabricksCreateJobsOperator | ||
| ============================ | ||
|
|
||
| Use the :class:`~airflow.providers.databricks.operators.DatabricksCreateJobsOperator` to create | ||
| (or reset) a Databricks job. This operator relies on past XComs to remember the ``job_id`` that | ||
| was created so that repeated calls with this operator will update the existing job rather than | ||
| creating new ones. When paired with the DatabricksRunNowOperator all runs will fall under the same | ||
| job within the Databricks UI. | ||
|
|
||
|
|
||
| Using the Operator | ||
| ------------------ | ||
|
|
||
| There are three ways to instantiate this operator. In the first way, you can take the JSON payload that you typically use | ||
| to call the ``api/2.1/jobs/create`` endpoint and pass it directly to our ``DatabricksCreateJobsOperator`` through the | ||
| ``json`` parameter. With this approach you get full control over the underlying payload to Jobs REST API, including | ||
| execution of Databricks jobs with multiple tasks, but it's harder to detect errors because of the lack of the type checking. | ||
|
|
||
| The second way to accomplish the same thing is to use the named parameters of the ``DatabricksCreateJobsOperator`` directly. Note that there is exactly | ||
| one named parameter for each top level parameter in the ``api/2.1/jobs/create`` endpoint. | ||
|
|
||
| The third way is to use both the json parameter **AND** the named parameters. They will be merged | ||
| together. If there are conflicts during the merge, the named parameters will take precedence and | ||
| override the top level ``json`` keys. | ||
|
|
||
| Currently the named parameters that ``DatabricksCreateJobsOperator`` supports are: | ||
| - ``name`` | ||
| - ``tags`` | ||
| - ``tasks`` | ||
| - ``job_clusters`` | ||
| - ``email_notifications`` | ||
| - ``webhook_notifications`` | ||
| - ``timeout_seconds`` | ||
| - ``schedule`` | ||
| - ``max_concurrent_runs`` | ||
| - ``git_source`` | ||
| - ``access_control_list`` | ||
stikkireddy marked this conversation as resolved.
Outdated
Show resolved
Hide resolved
|
||
|
|
||
|
|
||
| Examples | ||
| -------- | ||
|
|
||
| Specifying parameters as JSON | ||
| ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ | ||
|
|
||
| An example usage of the DatabricksCreateJobsOperator is as follows: | ||
|
|
||
| .. exampleinclude:: /../../tests/system/providers/databricks/example_databricks.py | ||
| :language: python | ||
| :start-after: [START howto_operator_databricks_jobs_create_json] | ||
| :end-before: [END howto_operator_databricks_jobs_create_json] | ||
|
|
||
| Using named parameters | ||
| ^^^^^^^^^^^^^^^^^^^^^^ | ||
|
|
||
| You can also use named parameters to initialize the operator and run the job. | ||
|
|
||
| .. exampleinclude:: /../../tests/system/providers/databricks/example_databricks.py | ||
| :language: python | ||
| :start-after: [START howto_operator_databricks_jobs_create_named] | ||
| :end-before: [END howto_operator_databricks_jobs_create_named] | ||
|
|
||
| Pairing with DatabricksRunNowOperator | ||
| ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ | ||
|
|
||
| You can use the ``job_id`` that is returned by the DatabricksCreateJobsOperator in the | ||
| return_value XCom as an argument to the DatabricksRunNowOperator to run the job. | ||
|
|
||
| .. exampleinclude:: /../../tests/system/providers/databricks/example_databricks.py | ||
| :language: python | ||
| :start-after: [START howto_operator_databricks_run_now] | ||
| :end-before: [END howto_operator_databricks_run_now] | ||
Uh oh!
There was an error while loading. Please reload this page.