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/19 08:26:51 UTC

[GitHub] [airflow] phanikumv opened a new pull request, #26475: Add `DbtCloudListJobsOperator`

phanikumv opened a new pull request, #26475:
URL: https://github.com/apache/airflow/pull/26475

   This PR adds the implementation for a `DbtCloudListJobsOperator` which retrieves job info from a given dbt Cloud account.
   
   <!--
   Thank you for contributing! Please make sure that your code changes
   are covered with tests. And in case of new features or big changes
   remember to adjust the documentation.
   
   Feel free to ping committers for the review!
   
   In case of an existing issue, reference it using one of the following:
   
   closes: #ISSUE
   related: #ISSUE
   
   How to write a good git commit message:
   http://chris.beams.io/posts/git-commit/
   -->
   
   ---
   **^ Add meaningful description above**
   
   Read the **[Pull Request Guidelines](https://github.com/apache/airflow/blob/main/CONTRIBUTING.rst#pull-request-guidelines)** for more information.
   In case of fundamental code changes, an Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvement+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in a newsfragment file, named `{pr_number}.significant.rst` or `{issue_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


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


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

Posted by GitBox <gi...@apache.org>.
potiuk commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r974294966


##########
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),
+        )
+        for job in list_jobs_response:
+            self.log.info(job.json()['data'])

Review Comment:
   Should not we return the job id list? This can be stored in an XCom value and something useful can be done with it (for example you coud pass it to a Dynamically mapped task to wait for the jobs to complete and do something with them?



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


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

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r976421781


##########
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:
   done



##########
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:
   done



##########
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:
   `account_id` can be retrieved through the decorator(from the connection object) or passed in the DAG. When retrieved through the decorator, self.account_id is not set, leading to the below log. 
   ```
   [2022-09-21, 11:14:05 UTC] {dbt.py:269} INFO - Jobs in the specified dbt Cloud account None are:
   [2022-09-21, 11:14:05 UTC] {dbt.py:280} INFO - 128688
   [2022-09-21, 11:14:05 UTC] {dbt.py:280} INFO - 128786
   ```
   Hence I made the log statement to `"Jobs in the specified dbt Cloud account are:"`
   



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


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

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r976423629


##########
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),
+        )
+        for job in list_jobs_response:
+            self.log.info(job.json()['data'])

Review Comment:
   This is addressed



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


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

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r978953353


##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,57 @@ 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: Optional. If an account ID is not provided explicitly,
+        the account ID from the dbt Cloud connection will be used.
+    :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 | None = None,
+        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
+
+    def execute(self, context: Context) -> list:
+        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
+        )
+        self.log.info(
+            "Jobs in the specified dbt Cloud account are:",
+        )
+        buffer = []
+        for job_metadata in list_jobs_response:
+            for job in job_metadata.json()['data']:
+                self.log.info(str(job["id"]))
+                buffer.append(job["id"])

Review Comment:
   done with this change in the latest commit 74606ca



##########
tests/providers/dbt/cloud/operators/test_dbt_cloud.py:
##########
@@ -388,3 +389,27 @@ def test_get_text_artifact_with_step(self, mock_get_artifact, conn_id, account_i
             account_id=account_id,
             step=2,
         )
+
+
+class TestDbtCloudListJobsOperator:
+    def setup_method(self):
+        self.dag = DAG("test_dbt_cloud_list_jobs_op", start_date=DEFAULT_DATE)
+        self.mock_ti = MagicMock()
+        self.mock_context = {"ti": self.mock_ti}
+
+    @patch("airflow.providers.dbt.cloud.hooks.dbt.DbtCloudHook.list_jobs")
+    @pytest.mark.parametrize(

Review Comment:
   done with this change in the latest commit 74606ca



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


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

Posted by GitBox <gi...@apache.org>.
josh-fell commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r977862168


##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,57 @@ 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: Optional. If an account ID is not provided explicitly,
+        the account ID from the dbt Cloud connection will be used.
+    :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 | None = None,
+        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
+
+    def execute(self, context: Context) -> list:
+        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
+        )
+        self.log.info(
+            "Jobs in the specified dbt Cloud account are:",
+        )
+        buffer = []
+        for job_metadata in list_jobs_response:
+            for job in job_metadata.json()['data']:
+                self.log.info(str(job["id"]))
+                buffer.append(job["id"])

Review Comment:
   WDYT about reversing these and calling `log.info` once with the list of job IDs?



##########
tests/providers/dbt/cloud/operators/test_dbt_cloud.py:
##########
@@ -388,3 +389,27 @@ def test_get_text_artifact_with_step(self, mock_get_artifact, conn_id, account_i
             account_id=account_id,
             step=2,
         )
+
+
+class TestDbtCloudListJobsOperator:
+    def setup_method(self):
+        self.dag = DAG("test_dbt_cloud_list_jobs_op", start_date=DEFAULT_DATE)
+        self.mock_ti = MagicMock()
+        self.mock_context = {"ti": self.mock_ti}
+
+    @patch("airflow.providers.dbt.cloud.hooks.dbt.DbtCloudHook.list_jobs")
+    @pytest.mark.parametrize(

Review Comment:
   Can you add a test where the `account_id` is not explicitly passed to the operator too?



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


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

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r978953421


##########
tests/providers/dbt/cloud/operators/test_dbt_cloud.py:
##########
@@ -388,3 +389,27 @@ def test_get_text_artifact_with_step(self, mock_get_artifact, conn_id, account_i
             account_id=account_id,
             step=2,
         )
+
+
+class TestDbtCloudListJobsOperator:
+    def setup_method(self):
+        self.dag = DAG("test_dbt_cloud_list_jobs_op", start_date=DEFAULT_DATE)
+        self.mock_ti = MagicMock()
+        self.mock_context = {"ti": self.mock_ti}
+
+    @patch("airflow.providers.dbt.cloud.hooks.dbt.DbtCloudHook.list_jobs")
+    @pytest.mark.parametrize(

Review Comment:
   done with this change in the latest commit



##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,57 @@ 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: Optional. If an account ID is not provided explicitly,
+        the account ID from the dbt Cloud connection will be used.
+    :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 | None = None,
+        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
+
+    def execute(self, context: Context) -> list:
+        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
+        )
+        self.log.info(
+            "Jobs in the specified dbt Cloud account are:",
+        )
+        buffer = []
+        for job_metadata in list_jobs_response:
+            for job in job_metadata.json()['data']:
+                self.log.info(str(job["id"]))
+                buffer.append(job["id"])

Review Comment:
   done with this change in the latest commit



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


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

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r974388755


##########
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),
+        )
+        for job in list_jobs_response:
+            self.log.info(job.json()['data'])

Review Comment:
   Yes makes sense @potiuk .  I will make the changes accordingly.



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


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

Posted by GitBox <gi...@apache.org>.
josh-fell commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r978980261


##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,54 @@ 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: Optional. If an account ID is not provided explicitly,
+        the account ID from the dbt Cloud connection will be used.
+    :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 | None = None,
+        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
+
+    def execute(self, context: Context) -> list:
+        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
+        )
+        buffer = []
+        for job_metadata in list_jobs_response:
+            for job in job_metadata.json()['data']:
+                buffer.append(job["id"])
+        self.log.info(f"Jobs in the specified dbt Cloud account are: {' '.join(map(str, buffer))}")

Review Comment:
   ```suggestion
           self.log.info("Jobs in the specified dbt Cloud account are: %s", ", ".join(map(str, buffer)))
   ```
   We should avoid using f-strings in logging calls.



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


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

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r979577060


##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,54 @@ 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: Optional. If an account ID is not provided explicitly,
+        the account ID from the dbt Cloud connection will be used.
+    :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 | None = None,
+        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
+
+    def execute(self, context: Context) -> list:
+        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
+        )
+        buffer = []
+        for job_metadata in list_jobs_response:
+            for job in job_metadata.json()['data']:
+                buffer.append(job["id"])
+        self.log.info(f"Jobs in the specified dbt Cloud account are: {' '.join(map(str, buffer))}")

Review Comment:
   fixed



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


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

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r976421965


##########
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:
   done



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


[GitHub] [airflow] josh-fell merged pull request #26475: Add `DbtCloudListJobsOperator`

Posted by GitBox <gi...@apache.org>.
josh-fell merged PR #26475:
URL: https://github.com/apache/airflow/pull/26475


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


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

Posted by GitBox <gi...@apache.org>.
josh-fell commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r975504025


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



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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r976421576


##########
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:
   made the change.



##########
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:
   fixed



##########
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:
   fixed



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


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

Posted by GitBox <gi...@apache.org>.
josh-fell commented on code in PR #26475:
URL: https://github.com/apache/airflow/pull/26475#discussion_r978980261


##########
airflow/providers/dbt/cloud/operators/dbt.py:
##########
@@ -218,3 +218,54 @@ 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: Optional. If an account ID is not provided explicitly,
+        the account ID from the dbt Cloud connection will be used.
+    :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 | None = None,
+        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
+
+    def execute(self, context: Context) -> list:
+        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
+        )
+        buffer = []
+        for job_metadata in list_jobs_response:
+            for job in job_metadata.json()['data']:
+                buffer.append(job["id"])
+        self.log.info(f"Jobs in the specified dbt Cloud account are: {' '.join(map(str, buffer))}")

Review Comment:
   ```suggestion
           self.log.info(f"Jobs in the specified dbt Cloud account are: %s", ", ".join(map(str, buffer)))
   ```
   We should avoid using f-strings in logging calls.



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