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/03/05 14:12:05 UTC

[GitHub] [airflow] alexott commented on a diff in pull request #29790: Provider Databricks add jobs create operator.

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


##########
airflow/providers/databricks/operators/databricks.py:
##########
@@ -151,6 +151,145 @@ 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[object] | None = None,
+        job_clusters: list[object] | None = None,
+        email_notifications: object | None = None,
+        webhook_notifications: object | None = None,
+        timeout_seconds: int | None = None,
+        schedule: dict[str, str] | None = None,
+        max_concurrent_runs: int | None = None,
+        git_source: dict[str, str] | None = None,
+        access_control_list: list[dict[str, str]] | None = None,

Review Comment:
   For new operators I would really like to use real data structures (for example, data classes), not the simple objects as it doesn't provide users with auto-completion, etc. - it's easy to make mistake in the untyped JSON definition 



##########
airflow/providers/databricks/operators/databricks.py:
##########
@@ -151,6 +151,145 @@ 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[object] | None = None,
+        job_clusters: list[object] | None = None,
+        email_notifications: object | None = None,
+        webhook_notifications: object | None = None,
+        timeout_seconds: int | None = None,
+        schedule: dict[str, str] | None = None,
+        max_concurrent_runs: int | None = None,
+        git_source: dict[str, str] | None = None,
+        access_control_list: list[dict[str, str]] | 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 ``DatabricksJobsCreateOperator``."""
+        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:
+            self.json["name"] = name
+        if tags is not None:
+            self.json["tags"] = tags
+        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 self._get_hook(caller="DatabricksJobsCreateOperator")
+
+    def _get_hook(self, caller: str) -> DatabricksHook:
+        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=caller,
+        )
+
+    def execute(self, context: Context) -> int:
+        self.job_id = self.xcom_pull(
+            context,
+            task_ids=self.task_id,
+            include_prior_dates=True,
+        )
+        if self.job_id:
+            self._hook.reset(self.job_id, self.json)

Review Comment:
   Instead of I would recommend to get the current job definition and compare with the new definition, and reset only if this definition changes (that should happen relatively rare). 
   
   Also, you need to handle following edge cases:
   
   * Job is deleted via UI - the reset will fail because job ID doesn't exist
   * We can lose XCom information, so we'll create a duplicate for the job.



-- 
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