You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/09/20 15:18:03 UTC

[GitHub] [airflow] josh-fell commented on a diff in pull request #26475: Add `DbtCloudListJobsOperator`

josh-fell commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r975492899


##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,55 @@ def execute(self, context: Context) -> None:
                 json.dump(response.json(), file)
             else:
                 file.write(response.text)
+
+
+class DbtCloudListJobsOperator(BaseOperator):
+    """
+    List jobs in a dbt Cloud project.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:DbtCloudListJobsOperator`
+
+    Retrieves metadata for all jobs tied to a specified dbt Cloud account. If a ``project_id`` is
+    supplied, only jobs pertaining to this project id will be retrieved.
+
+    :param account_id: Required. The ID of a dbt Cloud account.
+    :param order_by: Optional. Field to order the result by. Use '-' to indicate reverse order.
+        For example, to use reverse order by the run ID use ``order_by=-id``.
+    :param project_id: Optional. The ID of a dbt Cloud project.
+    """
+
+    template_fields = (
+        "account_id",
+        "project_id",
+    )
+
+    def __init__(
+        self,
+        *,
+        dbt_cloud_conn_id: str = DbtCloudHook.default_conn_name,
+        account_id: int,
+        project_id: int | None = None,
+        order_by: str | None = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.account_id = account_id
+        self.project_id = project_id
+        self.order_by = order_by
+        self.hook: DbtCloudHook

Review Comment:
   ```suggestion
   ```
   This doesn't need to be an instance attr of the operator since it's only called within the `execute()` method.



##########
airflow/providers/dbt/cloud/hooks/dbt.py:
##########
@@ -247,15 +247,15 @@ def get_project(self, project_id: int, account_id: int | None = None) -> Respons
     @fallback_to_default_account
     def list_jobs(
         self,
-        account_id: int | None = None,
+        account_id: int,

Review Comment:
   This should still be an optional parameter. The idea was that users most likely are only interacting with one dbt Cloud account and [can simply put that info in the Airflow Connection without having to pass it to every operator and/or hook method they wish to use](https://airflow.apache.org/docs/apache-airflow-providers-dbt-cloud/stable/connections.html#configuring-the-connection). The decorator on this method and others should retrieve the `account_id` from the Connection if one is not supplied explicitly (as an override from the Connection value for example).



##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,55 @@ def execute(self, context: Context) -> None:
                 json.dump(response.json(), file)
             else:
                 file.write(response.text)
+
+
+class DbtCloudListJobsOperator(BaseOperator):
+    """
+    List jobs in a dbt Cloud project.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:DbtCloudListJobsOperator`
+
+    Retrieves metadata for all jobs tied to a specified dbt Cloud account. If a ``project_id`` is
+    supplied, only jobs pertaining to this project id will be retrieved.
+
+    :param account_id: Required. The ID of a dbt Cloud account.
+    :param order_by: Optional. Field to order the result by. Use '-' to indicate reverse order.
+        For example, to use reverse order by the run ID use ``order_by=-id``.
+    :param project_id: Optional. The ID of a dbt Cloud project.
+    """
+
+    template_fields = (
+        "account_id",
+        "project_id",
+    )
+
+    def __init__(
+        self,
+        *,
+        dbt_cloud_conn_id: str = DbtCloudHook.default_conn_name,
+        account_id: int,
+        project_id: int | None = None,
+        order_by: str | None = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.account_id = account_id
+        self.project_id = project_id
+        self.order_by = order_by
+        self.hook: DbtCloudHook
+
+    def execute(self, context: Context) -> None:
+        self.hook = DbtCloudHook(self.dbt_cloud_conn_id)
+        list_jobs_response = self.hook.list_jobs(
+            account_id=self.account_id, order_by=self.order_by, project_id=self.project_id
+        )
+        self.log.info(
+            "Jobs in the specified dbt Cloud account %s and project %s are:",

Review Comment:
   WDYT about having having either separate logging messages: one for when `project_id` is None and another when it's provided? IMO it seems a little odd to have a message of `... dbt Cloud account 12435 and project None are: ...` when a `project_id` isn't provided.



##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,55 @@ def execute(self, context: Context) -> None:
                 json.dump(response.json(), file)
             else:
                 file.write(response.text)
+
+
+class DbtCloudListJobsOperator(BaseOperator):
+    """
+    List jobs in a dbt Cloud project.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:DbtCloudListJobsOperator`
+
+    Retrieves metadata for all jobs tied to a specified dbt Cloud account. If a ``project_id`` is
+    supplied, only jobs pertaining to this project id will be retrieved.
+
+    :param account_id: Required. The ID of a dbt Cloud account.
+    :param order_by: Optional. Field to order the result by. Use '-' to indicate reverse order.
+        For example, to use reverse order by the run ID use ``order_by=-id``.
+    :param project_id: Optional. The ID of a dbt Cloud project.
+    """
+
+    template_fields = (
+        "account_id",
+        "project_id",
+    )
+
+    def __init__(
+        self,
+        *,
+        dbt_cloud_conn_id: str = DbtCloudHook.default_conn_name,
+        account_id: int,
+        project_id: int | None = None,
+        order_by: str | None = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.account_id = account_id
+        self.project_id = project_id
+        self.order_by = order_by
+        self.hook: DbtCloudHook
+
+    def execute(self, context: Context) -> None:
+        self.hook = DbtCloudHook(self.dbt_cloud_conn_id)
+        list_jobs_response = self.hook.list_jobs(
+            account_id=self.account_id, order_by=self.order_by, project_id=self.project_id
+        )
+        self.log.info(
+            "Jobs in the specified dbt Cloud account %s and project %s are:",
+            str(self.account_id),
+            str(self.project_id),

Review Comment:
   ```suggestion
               self.account_id,
               self.project_id,
   ```
   The string conversion should be handled implicitly in logging.



##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,55 @@ def execute(self, context: Context) -> None:
                 json.dump(response.json(), file)
             else:
                 file.write(response.text)
+
+
+class DbtCloudListJobsOperator(BaseOperator):
+    """
+    List jobs in a dbt Cloud project.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:DbtCloudListJobsOperator`
+
+    Retrieves metadata for all jobs tied to a specified dbt Cloud account. If a ``project_id`` is
+    supplied, only jobs pertaining to this project id will be retrieved.
+
+    :param account_id: Required. The ID of a dbt Cloud account.
+    :param order_by: Optional. Field to order the result by. Use '-' to indicate reverse order.
+        For example, to use reverse order by the run ID use ``order_by=-id``.
+    :param project_id: Optional. The ID of a dbt Cloud project.
+    """
+
+    template_fields = (
+        "account_id",
+        "project_id",
+    )
+
+    def __init__(
+        self,
+        *,
+        dbt_cloud_conn_id: str = DbtCloudHook.default_conn_name,
+        account_id: int,

Review Comment:
   ```suggestion
           account_id: int | None = None,
   ```



##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,55 @@ def execute(self, context: Context) -> None:
                 json.dump(response.json(), file)
             else:
                 file.write(response.text)
+
+
+class DbtCloudListJobsOperator(BaseOperator):
+    """
+    List jobs in a dbt Cloud project.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:DbtCloudListJobsOperator`
+
+    Retrieves metadata for all jobs tied to a specified dbt Cloud account. If a ``project_id`` is
+    supplied, only jobs pertaining to this project id will be retrieved.
+
+    :param account_id: Required. The ID of a dbt Cloud account.

Review Comment:
   ```suggestion
       :param account_id: Optional. The ID of a dbt Cloud account. If an account ID is not provided explicitly, the account ID from the dbt Cloud connection will be used.
   ```



##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,55 @@ def execute(self, context: Context) -> None:
                 json.dump(response.json(), file)
             else:
                 file.write(response.text)
+
+
+class DbtCloudListJobsOperator(BaseOperator):
+    """
+    List jobs in a dbt Cloud project.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:DbtCloudListJobsOperator`
+
+    Retrieves metadata for all jobs tied to a specified dbt Cloud account. If a ``project_id`` is
+    supplied, only jobs pertaining to this project id will be retrieved.
+
+    :param account_id: Required. The ID of a dbt Cloud account.
+    :param order_by: Optional. Field to order the result by. Use '-' to indicate reverse order.
+        For example, to use reverse order by the run ID use ``order_by=-id``.
+    :param project_id: Optional. The ID of a dbt Cloud project.
+    """
+
+    template_fields = (
+        "account_id",
+        "project_id",
+    )
+
+    def __init__(
+        self,
+        *,
+        dbt_cloud_conn_id: str = DbtCloudHook.default_conn_name,
+        account_id: int,
+        project_id: int | None = None,
+        order_by: str | None = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.account_id = account_id
+        self.project_id = project_id
+        self.order_by = order_by
+        self.hook: DbtCloudHook
+
+    def execute(self, context: Context) -> None:
+        self.hook = DbtCloudHook(self.dbt_cloud_conn_id)
+        list_jobs_response = self.hook.list_jobs(
+            account_id=self.account_id, order_by=self.order_by, project_id=self.project_id
+        )

Review Comment:
   ```suggestion
           hook = DbtCloudHook(self.dbt_cloud_conn_id)
           list_jobs_response = hook.list_jobs(
               account_id=self.account_id, order_by=self.order_by, project_id=self.project_id
           )
   ```
   Related to above comment.



##########
airflow/providers/dbt/cloud/hooks/dbt.py:
##########
@@ -247,15 +247,15 @@ def get_project(self, project_id: int, account_id: int | None = None) -> Respons
     @fallback_to_default_account
     def list_jobs(
         self,
-        account_id: int | None = None,
+        account_id: int,
         order_by: str | None = None,
         project_id: int | None = None,
     ) -> list[Response]:
         """
         Retrieves metadata for all jobs tied to a specified dbt Cloud account. If a ``project_id`` is
-        supplied, only jobs pertaining to this job will be retrieved.
+        supplied, only jobs pertaining to this project will be retrieved.

Review Comment:
   Nice catch!



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