You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "alexott (via GitHub)" <gi...@apache.org> on 2023/07/06 17:24:31 UTC

[GitHub] [airflow] alexott commented on a diff in pull request #32221: Add `DatabricksJobsCreateOperator`

alexott commented on code in PR #32221:
URL: https://github.com/apache/airflow/pull/32221#discussion_r1254328800


##########
airflow/providers/databricks/hooks/databricks.py:
##########
@@ -135,9 +137,27 @@ def __init__(
     ) -> None:
         super().__init__(databricks_conn_id, timeout_seconds, retry_limit, retry_delay, retry_args, caller)
 
+    def create(self, json: dict) -> int:

Review Comment:
   would be useful to call this `job_create`



##########
docs/apache-airflow-providers-databricks/operators/jobs_create.rst:
##########
@@ -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.
+
+
+
+DatabricksJobsCreateOperator
+============================
+
+Use the :class:`~airflow.providers.databricks.operators.DatabricksJobsCreateOperator` 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 ``DatabricksJobsCreateOperator`` 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 ``DatabricksJobsCreateOperator`` 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 ``DatabricksJobsCreateOperator`` supports are:
+  - ``name``
+  - ``tags``
+  - ``tasks``
+  - ``job_clusters``
+  - ``email_notifications``
+  - ``webhook_notifications``
+  - ``timeout_seconds``
+  - ``schedule``
+  - ``max_concurrent_runs``
+  - ``git_source``
+  - ``access_control_list``

Review Comment:
   check here as well



##########
airflow/providers/databricks/operators/databricks.py:
##########
@@ -160,6 +162,142 @@ def get_link(
         return XCom.get_value(key=XCOM_RUN_PAGE_URL_KEY, ti_key=ti_key)
 
 
+class DatabricksJobsCreateOperator(BaseOperator):
+    """
+    Creates (or resets) a Databricks job using the
+    `api/2.1/jobs/create
+    <https://docs.databricks.com/dev-tools/api/latest/jobs.html#operation/JobsCreate>`_
+    (or `api/2.1/jobs/reset
+    <https://docs.databricks.com/dev-tools/api/latest/jobs.html#operation/JobsReset>`_)
+    API endpoint.
+
+    .. seealso::
+        https://docs.databricks.com/dev-tools/api/latest/jobs.html#operation/JobsCreate
+
+    :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[j.JobTaskSettings] | None = None,
+        job_clusters: list[j.JobCluster] | None = None,
+        email_notifications: j.JobEmailNotifications | None = None,
+        webhook_notifications: j.JobWebhookNotifications | None = None,
+        timeout_seconds: int | None = None,
+        schedule: j.CronSchedule | None = None,
+        max_concurrent_runs: int | None = None,
+        git_source: j.GitSource | None = None,
+        access_control_list: list[j.AccessControlRequest] | None = None,

Review Comment:
   We need to double check with the dev team - but I think that it's a bug in the API spec. We have access_control_list only for SubmitRun, not for jobs - we use permissions API for controlling the permissions of jobs visible in UI.



##########
airflow/providers/databricks/hooks/databricks.py:
##########
@@ -135,9 +137,27 @@ def __init__(
     ) -> None:
         super().__init__(databricks_conn_id, timeout_seconds, retry_limit, retry_delay, retry_args, caller)
 
+    def create(self, json: dict) -> int:
+        """
+        Utility function to call the ``api/2.1/jobs/create`` endpoint.
+
+        :param json: The data used in the body of the request to the ``create`` endpoint.
+        :return: the job_id as an int
+        """
+        response = self._do_api_call(CREATE_ENDPOINT, json)
+        return response["job_id"]
+
+    def reset(self, job_id: str, json: dict):

Review Comment:
   Same here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org