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/01/20 20:40:19 UTC

[GitHub] [airflow] josh-fell opened a new pull request #20998: Add dbt Cloud provider

josh-fell opened a new pull request #20998:
URL: https://github.com/apache/airflow/pull/20998


   This PR adds a new provider to interface with [dbt Cloud](https://docs.getdbt.com/docs/dbt-cloud/cloud-overview) via the [dbt Cloud API](https://docs.getdbt.com/dbt-cloud/api-v2).  The provider includes:
   
   - `DbtCloudHook` which implements an abstraction for almost all of the available endpoints in the dbt Cloud API and inherits from the existing `HttpHook`.
   - Two operators, `DbtCloudRunJobOperator` and `DbtCloudGetJobRunArtifactOperator`, which triggers a dbt Cloud job and downloads a run artifact, respectively.
   - A sensor, `DbtCloudJobRunSensor`, to poll status of a specific dbt Cloud job run.
   - An operator link to allow users to navigate directly to the triggered dbt Cloud job run.
   - A `test_connection()` method in the `DbtCloudHook` for users to test connections prior to executing DAGs.
   
   ---
   **^ 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 change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+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 [UPDATING.md](https://github.com/apache/airflow/blob/main/UPDATING.md).
   


-- 
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] mik-laj commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r789235758



##########
File path: airflow/providers/dbt/cloud/operators/dbt.py
##########
@@ -0,0 +1,203 @@
+# 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.
+
+import json
+from typing import TYPE_CHECKING, Any, Dict, List, Optional
+
+from airflow.models import BaseOperator, BaseOperatorLink, TaskInstance
+from airflow.providers.dbt.cloud.hooks.dbt import DbtCloudHook, DbtCloudJobRunException, DbtCloudJobRunStatus
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class DbtCloudRunJobOperatorLink(BaseOperatorLink):
+    """
+    Operator link for DbtCloudRunJobOperator. This link allows users to monitor the triggered job run
+    directly in dbt Cloud.
+    """
+
+    name = "Monitor Job Run"
+
+    def get_link(self, operator, dttm):
+        ti = TaskInstance(task=operator, execution_date=dttm)
+        job_run_url = ti.xcom_pull(task_ids=operator.task_id, key="job_run_url")
+
+        return job_run_url
+
+
+class DbtCloudRunJobOperator(BaseOperator):
+    """
+    Executes a dbt Cloud job.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:DbtCloudRunJobOperator`
+
+    :param dbt_cloud_conn_id: The connection ID for connecting to dbt Cloud.
+    :param job_id: The ID of a dbt Cloud job.
+    :param account_id: Optional. The ID of a dbt Cloud account.
+    :param trigger_reason: Optional. Description of the reason to trigger the job.
+    :param steps_override: Optional. List of dbt commands to execute when triggering the job instead of those
+        configured in dbt Cloud.
+    :param schema_override: Optional. Override the destination schema in the configured target for this job.
+    :param wait_for_termination: Flag to wait on a job run's termination.  By default, this feature is
+        enabled but could be disabled to perform an asynchronous wait for a long-running job run execution
+        using the ``DbtCloudJobRunSensor``.
+    :param timeout: Time in seconds to wait for a job run to reach a terminal status for non-asynchronous
+        waits. Used only if ``wait_for_termination`` is True. Defaults to 7 days.
+    :param check_interval: Time in seconds to check on a job run's status for non-asynchronous waits.
+        Used only if ``wait_for_termination`` is True. Defaults to 60 seconds.
+    :param additional_run_config: Optional. Any additional parameters that should be included in the API
+        request when triggering the job.
+    :return: The ID of the triggered dbt Cloud job run.
+    """
+
+    template_fields = ("dbt_cloud_conn_id", "job_id", "account_id", "trigger_reason")
+
+    operator_extra_links = (DbtCloudRunJobOperatorLink(),)
+
+    def __init__(
+        self,
+        *,
+        dbt_cloud_conn_id: str = DbtCloudHook.default_conn_name,
+        job_id: int,
+        account_id: Optional[int] = None,
+        trigger_reason: Optional[str] = None,
+        steps_override: Optional[List[str]] = None,
+        schema_override: Optional[str] = None,
+        wait_for_termination: bool = True,
+        timeout: int = 60 * 60 * 24 * 7,
+        check_interval: int = 60,
+        additional_run_config: Optional[Dict[str, Any]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.hook = DbtCloudHook(self.dbt_cloud_conn_id)

Review comment:
       We should avoid initializing hook in ctor. 




-- 
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] subkanthi commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
subkanthi commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r790159708



##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,488 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+
+        return request
+
+
+class JobRunInfo(TypedDict):
+    """Type class for the ``job_run_info`` dictionary."""
+
+    account_id: int
+    run_id: int
+
+
+class DbtCloudJobRunStatus(Enum):
+    """dbt Cloud Job statuses."""
+
+    QUEUED = 1
+    STARTING = 2
+    RUNNING = 3
+    SUCCESS = 10
+    ERROR = 20
+    CANCELLED = 30
+    TERMINAL_STATUSES = (SUCCESS, ERROR, CANCELLED)
+
+    @classmethod
+    def check_is_valid(cls, statuses: Union[int, Sequence[int], Set[int]]):
+        """Validates input statuses are a known value."""
+        if isinstance(statuses, (Sequence, Set)):
+            for status in statuses:
+                cls(status)
+        else:
+            cls(statuses)
+
+    @classmethod
+    def is_terminal(cls, status: int) -> bool:
+        """Checks if the input status is that of a terminal type."""
+        cls.check_is_valid(statuses=status)
+
+        return status in cls.TERMINAL_STATUSES.value
+
+
+class DbtCloudJobRunException(AirflowException):
+    """An exception that indicates a job run failed to complete."""
+
+
+class DbtCloudHook(HttpHook):
+    """
+    Interact with dbt Cloud using the V2 API.
+
+    :param dbt_cloud_conn_id: The ID of the :ref:`dbt Cloud connection <howto/connection:dbt-cloud>`.
+    """
+
+    conn_name_attr = "dbt_cloud_conn_id"
+    default_conn_name = "dbt_cloud_default"
+    conn_type = "dbt_cloud"
+    hook_name = "dbt Cloud"
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict[str, Any]:
+        """Builds custom field behavior for the dbt Cloud connection form in the Airflow UI."""
+        return {
+            "hidden_fields": ["host", "port", "schema", "extra"],
+            "relabeling": {"login": "Account ID", "password": "API Token"},
+        }
+
+    def __init__(self, dbt_cloud_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(auth_type=TokenAuth)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.base_url = "https://cloud.getdbt.com/api/v2/accounts/"
+
+    @cached_property
+    def conn(self) -> Connection:
+        _conn = self.get_connection(self.dbt_cloud_conn_id)
+        if not _conn.password:
+            raise AirflowException("An API token is required to connect to dbt Cloud.")
+
+        return _conn
+
+    def get_conn(self, *args, **kwargs) -> Session:
+        session = Session()
+        session.auth = self.auth_type(self.conn.password)
+
+        return session
+
+    def _paginate(self, endpoint: str, payload: Optional[Dict[str, Any]] = None) -> List[Response]:
+        results = []
+        response = self.run(endpoint=endpoint, data=payload)
+        resp_json = response.json()
+        limit = resp_json["extra"]["filters"]["limit"]
+        num_total_results = resp_json["extra"]["pagination"]["total_count"]
+        num_current_results = resp_json["extra"]["pagination"]["count"]
+        results.append(response)
+
+        if not num_current_results == num_total_results:
+            _paginate_payload = payload.copy() if payload else {}
+            _paginate_payload["offset"] = limit
+
+            while True:
+                if num_current_results < num_total_results:
+                    response = self.run(endpoint=endpoint, data=_paginate_payload)
+                    resp_json = response.json()
+                    if resp_json["data"]:

Review comment:
       would it help to have log in the "else" if data is not present in the response.




-- 
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] subkanthi commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
subkanthi commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r790145664



##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,488 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+
+        return request
+
+
+class JobRunInfo(TypedDict):
+    """Type class for the ``job_run_info`` dictionary."""
+
+    account_id: int
+    run_id: int
+
+
+class DbtCloudJobRunStatus(Enum):
+    """dbt Cloud Job statuses."""
+
+    QUEUED = 1
+    STARTING = 2
+    RUNNING = 3
+    SUCCESS = 10
+    ERROR = 20
+    CANCELLED = 30
+    TERMINAL_STATUSES = (SUCCESS, ERROR, CANCELLED)
+
+    @classmethod
+    def check_is_valid(cls, statuses: Union[int, Sequence[int], Set[int]]):
+        """Validates input statuses are a known value."""
+        if isinstance(statuses, (Sequence, Set)):
+            for status in statuses:
+                cls(status)
+        else:
+            cls(statuses)
+
+    @classmethod
+    def is_terminal(cls, status: int) -> bool:
+        """Checks if the input status is that of a terminal type."""
+        cls.check_is_valid(statuses=status)
+
+        return status in cls.TERMINAL_STATUSES.value
+
+
+class DbtCloudJobRunException(AirflowException):
+    """An exception that indicates a job run failed to complete."""
+
+
+class DbtCloudHook(HttpHook):
+    """
+    Interact with dbt Cloud using the V2 API.
+
+    :param dbt_cloud_conn_id: The ID of the :ref:`dbt Cloud connection <howto/connection:dbt-cloud>`.
+    """
+
+    conn_name_attr = "dbt_cloud_conn_id"
+    default_conn_name = "dbt_cloud_default"
+    conn_type = "dbt_cloud"
+    hook_name = "dbt Cloud"
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict[str, Any]:
+        """Builds custom field behavior for the dbt Cloud connection form in the Airflow UI."""
+        return {
+            "hidden_fields": ["host", "port", "schema", "extra"],
+            "relabeling": {"login": "Account ID", "password": "API Token"},
+        }
+
+    def __init__(self, dbt_cloud_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(auth_type=TokenAuth)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.base_url = "https://cloud.getdbt.com/api/v2/accounts/"
+
+    @cached_property
+    def conn(self) -> Connection:
+        _conn = self.get_connection(self.dbt_cloud_conn_id)
+        if not _conn.password:
+            raise AirflowException("An API token is required to connect to dbt Cloud.")
+
+        return _conn
+
+    def get_conn(self, *args, **kwargs) -> Session:
+        session = Session()
+        session.auth = self.auth_type(self.conn.password)
+
+        return session
+
+    def _paginate(self, endpoint: str, payload: Optional[Dict[str, Any]] = None) -> List[Response]:
+        results = []
+        response = self.run(endpoint=endpoint, data=payload)
+        resp_json = response.json()
+        limit = resp_json["extra"]["filters"]["limit"]
+        num_total_results = resp_json["extra"]["pagination"]["total_count"]
+        num_current_results = resp_json["extra"]["pagination"]["count"]
+        results.append(response)
+
+        if not num_current_results == num_total_results:
+            _paginate_payload = payload.copy() if payload else {}
+            _paginate_payload["offset"] = limit
+
+            while True:
+                if num_current_results < num_total_results:
+                    response = self.run(endpoint=endpoint, data=_paginate_payload)
+                    resp_json = response.json()
+                    if resp_json["data"]:
+                        results.append(response)
+                        num_current_results += resp_json["extra"]["pagination"]["count"]
+                        _paginate_payload["offset"] += limit
+                else:
+                    break
+
+        return results
+
+    def _run_and_get_response(
+        self,
+        method: str = "GET",
+        endpoint: Optional[str] = None,
+        payload: Union[str, Dict[str, Any], None] = None,
+        paginate: bool = False,
+    ) -> Any:
+        self.method = method
+
+        if paginate:
+            if isinstance(payload, str):
+                raise ValueError("Payload cannot be a string to paginate a response.")
+
+            if endpoint:
+                return self._paginate(endpoint=endpoint, payload=payload)
+            else:
+                raise ValueError("An endpoint is needed to paginate a response.")
+
+        return self.run(endpoint=endpoint, data=payload)
+
+    def list_accounts(self) -> List[Response]:
+        """
+        Retrieves all of the dbt Cloud accounts the configured API token is authorized to access.
+
+        :return: List of request responses.
+        """
+        return self._run_and_get_response()
+
+    @fallback_to_default_account
+    def get_account(self, account_id: Optional[int] = None) -> Response:
+        """
+        Retrieves metadata for a specific dbt Cloud account.
+
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :return: The request response.
+        """
+        return self._run_and_get_response(endpoint=f"{account_id}/")
+
+    @fallback_to_default_account
+    def list_projects(self, account_id: Optional[int] = None) -> List[Response]:
+        """
+        Retrieves metadata for all projects tied to a specified dbt Cloud account.
+
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :return: List of request responses.
+        """
+        return self._run_and_get_response(endpoint=f"{account_id}/projects/", paginate=True)
+
+    @fallback_to_default_account
+    def get_project(self, project_id: int, account_id: Optional[int] = None) -> Response:
+        """
+        Retrieves metadata for a specific project.
+
+        :param project_id: The ID of a dbt Cloud project.
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :return: The request response.
+        """
+        return self._run_and_get_response(endpoint=f"{account_id}/projects/{project_id}/")
+
+    @fallback_to_default_account
+    def list_jobs(
+        self,
+        account_id: Optional[int] = None,
+        order_by: Optional[str] = None,
+        project_id: Optional[int] = 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.
+
+        :param account_id: Optional. 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: The ID of a dbt Cloud project.
+        :return: List of request responses.
+        """
+        return self._run_and_get_response(
+            endpoint=f"{account_id}/jobs/",
+            payload={"order_by": order_by, "project_id": project_id},
+            paginate=True,
+        )
+
+    @fallback_to_default_account
+    def get_job(self, job_id: int, account_id: Optional[int] = None) -> Response:
+        """
+        Retrieves metadata for a specific job.
+
+        :param job_id: The ID of a dbt Cloud job.
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :return: The request response.
+        """
+        return self._run_and_get_response(endpoint=f"{account_id}/jobs/{job_id}")
+
+    @fallback_to_default_account
+    def trigger_job_run(
+        self,
+        job_id: int,
+        cause: str,
+        account_id: Optional[int] = None,
+        steps_override: Optional[List[str]] = None,
+        schema_override: Optional[str] = None,
+        additional_run_config: Optional[Dict[str, Any]] = None,
+    ) -> Response:
+        """
+        Triggers a run of a dbt Cloud job.
+
+        :param job_id: The ID of a dbt Cloud job.
+        :param cause: Description of the reason to trigger the job.
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :param steps_override: Optional. List of dbt commands to execute when triggering the job
+            instead of those configured in dbt Cloud.
+        :param schema_override: Optional. Override the destination schema in the configured target for this
+            job.
+        :param additional_run_config: Optional. Any additional parameters that should be included in the API
+            request when triggering the job.
+        :return: The request response.
+        """
+        if additional_run_config is None:
+            additional_run_config = {}
+
+        payload = {
+            "cause": cause,
+            "steps_override": steps_override,
+            "schema_override": schema_override,
+        }
+        payload.update(additional_run_config)
+
+        return self._run_and_get_response(
+            method="POST",
+            endpoint=f"{account_id}/jobs/{job_id}/run/",
+            payload=json.dumps(payload),
+        )
+
+    @fallback_to_default_account
+    def list_job_runs(
+        self,
+        account_id: Optional[int] = None,
+        include_related: Optional[List[str]] = None,
+        job_definition_id: Optional[int] = None,
+        order_by: Optional[str] = None,
+    ) -> List[Response]:
+        """
+        Retrieves metadata for all of the dbt Cloud job runs for an account. If a ``job_definition_id`` is
+        supplied, only metadata for runs of that specific job are pulled.
+
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :param include_related: Optional. List of related fields to pull with the run.
+            Valid values are "trigger", "job", "repository", and "environment".
+        :param job_definition_id: Optional. The dbt Cloud job ID to retrieve run metadata.
+        :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``.
+        :return: List of request responses.
+        """
+        return self._run_and_get_response(
+            endpoint=f"{account_id}/runs/",
+            payload={
+                "include_related": include_related,
+                "job_definition_id": job_definition_id,
+                "order_by": order_by,
+            },
+            paginate=True,
+        )
+
+    @fallback_to_default_account
+    def get_job_run(
+        self, run_id: int, account_id: Optional[int] = None, include_related: Optional[List[str]] = None
+    ) -> Response:
+        """
+        Retrieves metadata for a specific run of a dbt Cloud job.
+
+        :param run_id: The ID of a dbt Cloud job run.
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :param include_related: Optional. List of related fields to pull with the run.
+            Valid values are "trigger", "job", "repository", and "environment".
+        :return: The request response.
+        """
+        return self._run_and_get_response(
+            endpoint=f"{account_id}/runs/{run_id}/",

Review comment:
       Should we log an error if account_id is not passed, looks like it might construct the url with None.




-- 
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] subkanthi commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
subkanthi commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r790159708



##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,488 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+
+        return request
+
+
+class JobRunInfo(TypedDict):
+    """Type class for the ``job_run_info`` dictionary."""
+
+    account_id: int
+    run_id: int
+
+
+class DbtCloudJobRunStatus(Enum):
+    """dbt Cloud Job statuses."""
+
+    QUEUED = 1
+    STARTING = 2
+    RUNNING = 3
+    SUCCESS = 10
+    ERROR = 20
+    CANCELLED = 30
+    TERMINAL_STATUSES = (SUCCESS, ERROR, CANCELLED)
+
+    @classmethod
+    def check_is_valid(cls, statuses: Union[int, Sequence[int], Set[int]]):
+        """Validates input statuses are a known value."""
+        if isinstance(statuses, (Sequence, Set)):
+            for status in statuses:
+                cls(status)
+        else:
+            cls(statuses)
+
+    @classmethod
+    def is_terminal(cls, status: int) -> bool:
+        """Checks if the input status is that of a terminal type."""
+        cls.check_is_valid(statuses=status)
+
+        return status in cls.TERMINAL_STATUSES.value
+
+
+class DbtCloudJobRunException(AirflowException):
+    """An exception that indicates a job run failed to complete."""
+
+
+class DbtCloudHook(HttpHook):
+    """
+    Interact with dbt Cloud using the V2 API.
+
+    :param dbt_cloud_conn_id: The ID of the :ref:`dbt Cloud connection <howto/connection:dbt-cloud>`.
+    """
+
+    conn_name_attr = "dbt_cloud_conn_id"
+    default_conn_name = "dbt_cloud_default"
+    conn_type = "dbt_cloud"
+    hook_name = "dbt Cloud"
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict[str, Any]:
+        """Builds custom field behavior for the dbt Cloud connection form in the Airflow UI."""
+        return {
+            "hidden_fields": ["host", "port", "schema", "extra"],
+            "relabeling": {"login": "Account ID", "password": "API Token"},
+        }
+
+    def __init__(self, dbt_cloud_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(auth_type=TokenAuth)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.base_url = "https://cloud.getdbt.com/api/v2/accounts/"
+
+    @cached_property
+    def conn(self) -> Connection:
+        _conn = self.get_connection(self.dbt_cloud_conn_id)
+        if not _conn.password:
+            raise AirflowException("An API token is required to connect to dbt Cloud.")
+
+        return _conn
+
+    def get_conn(self, *args, **kwargs) -> Session:
+        session = Session()
+        session.auth = self.auth_type(self.conn.password)
+
+        return session
+
+    def _paginate(self, endpoint: str, payload: Optional[Dict[str, Any]] = None) -> List[Response]:
+        results = []
+        response = self.run(endpoint=endpoint, data=payload)
+        resp_json = response.json()
+        limit = resp_json["extra"]["filters"]["limit"]
+        num_total_results = resp_json["extra"]["pagination"]["total_count"]
+        num_current_results = resp_json["extra"]["pagination"]["count"]
+        results.append(response)
+
+        if not num_current_results == num_total_results:
+            _paginate_payload = payload.copy() if payload else {}
+            _paginate_payload["offset"] = limit
+
+            while True:
+                if num_current_results < num_total_results:
+                    response = self.run(endpoint=endpoint, data=_paginate_payload)
+                    resp_json = response.json()
+                    if resp_json["data"]:

Review comment:
       would it help to have a log  statement in the "else" if data is not present in the response.




-- 
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] sungchun12 edited a comment on pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
sungchun12 edited a comment on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1029093982


   @mik-laj thanks a bunch for the follow up. I'm in conversations internally with the dbt Labs engineers to specify a User-Agent in the headers of the dbt Cloud API requests for tracking.
   
   I'll submit a pull request for the dbt Labs docs after this is merged!
   
   @josh-fell I'll follow up with you personally on the above.


-- 
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 pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
josh-fell commented on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1050938645


   @mik-laj Are there any other nuances/features I should try to incorporate in the provider? The last suggestion was a great one.


-- 
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 pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
josh-fell commented on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1032664180


   > ... specify a User-Agent in the headers of the dbt Cloud API requests for tracking.
   
   @sungchun12 @mik-laj This is implemented now if you'd like to take a look.  I also refactored the operator link slightly to not create ad hoc TaskInstances to align with #21285.
   


-- 
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 change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r789725177



##########
File path: docs/apache-airflow-providers-dbt-cloud/connections.rst
##########
@@ -0,0 +1,66 @@
+ .. 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.
+
+
+
+.. _howto/connection:dbt-cloud:
+
+Connecting to dbt Cloud
+=======================
+
+After installing the dbt Cloud provider in your Airflow environment, the corresponding connection type of
+``dbt_cloud`` will be made available. The following describes how to configure an API token and optionally
+provide an Account ID for your dbt Cloud connection.
+
+Default Connection ID
+~~~~~~~~~~~~~~~~~~~~~
+
+All hooks and operators related to dbt Cloud use ``dbt_cloud_default`` by default.
+
+
+Authenticating to the dbt Cloud API
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+To interact with the dbt Cloud API in Airflow, either a
+`User API Token <https://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/user-tokens>`__ or a
+`Service Account API Token <hhttps://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/service-tokens>`__ is
+required.
+
+
+Configuring the connection
+~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Password (required)
+    The API token to use when authenticating to the dbt Cloud API.
+
+    If using the Connection form in the Airflow UI, the token can also be stored in the "API Token" field.
+
+Login (optional)
+    The Account ID to be used as the default Account ID for dbt Cloud operators or
+    :class:`~airflow.providers.dbt.cloud.hooks.dbt.DbtCloudHook` methods. If an Account ID is provided in the
+    connection, you are not required to pass ``account_id`` to operators or hook methods. The ``account_id``
+    value will be retrieved from the Airflow connection instead. If needed, the ``account_id`` can still be
+    explicitly passed to an operator or hook method as an override the default value configured in the
+    connection.
+
+    If using the Connection form in the Airflow UI, the Account ID can also be stored in the "Account ID"
+    field.
+
+    .. note::
+
+      If an Account ID is not provided in an Airflow connection, ``account_id`` *must* be explicitly passed to
+      an operator or hook method.

Review comment:
       Absolutely. Will add.




-- 
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 change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r790383368



##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,488 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+
+        return request
+
+
+class JobRunInfo(TypedDict):
+    """Type class for the ``job_run_info`` dictionary."""
+
+    account_id: int
+    run_id: int
+
+
+class DbtCloudJobRunStatus(Enum):
+    """dbt Cloud Job statuses."""
+
+    QUEUED = 1
+    STARTING = 2
+    RUNNING = 3
+    SUCCESS = 10
+    ERROR = 20
+    CANCELLED = 30
+    TERMINAL_STATUSES = (SUCCESS, ERROR, CANCELLED)
+
+    @classmethod
+    def check_is_valid(cls, statuses: Union[int, Sequence[int], Set[int]]):
+        """Validates input statuses are a known value."""
+        if isinstance(statuses, (Sequence, Set)):
+            for status in statuses:
+                cls(status)
+        else:
+            cls(statuses)
+
+    @classmethod
+    def is_terminal(cls, status: int) -> bool:
+        """Checks if the input status is that of a terminal type."""
+        cls.check_is_valid(statuses=status)
+
+        return status in cls.TERMINAL_STATUSES.value
+
+
+class DbtCloudJobRunException(AirflowException):
+    """An exception that indicates a job run failed to complete."""
+
+
+class DbtCloudHook(HttpHook):
+    """
+    Interact with dbt Cloud using the V2 API.
+
+    :param dbt_cloud_conn_id: The ID of the :ref:`dbt Cloud connection <howto/connection:dbt-cloud>`.
+    """
+
+    conn_name_attr = "dbt_cloud_conn_id"
+    default_conn_name = "dbt_cloud_default"
+    conn_type = "dbt_cloud"
+    hook_name = "dbt Cloud"
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict[str, Any]:
+        """Builds custom field behavior for the dbt Cloud connection form in the Airflow UI."""
+        return {
+            "hidden_fields": ["host", "port", "schema", "extra"],
+            "relabeling": {"login": "Account ID", "password": "API Token"},
+        }
+
+    def __init__(self, dbt_cloud_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(auth_type=TokenAuth)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.base_url = "https://cloud.getdbt.com/api/v2/accounts/"
+
+    @cached_property
+    def conn(self) -> Connection:
+        _conn = self.get_connection(self.dbt_cloud_conn_id)
+        if not _conn.password:
+            raise AirflowException("An API token is required to connect to dbt Cloud.")
+
+        return _conn
+
+    def get_conn(self, *args, **kwargs) -> Session:
+        session = Session()
+        session.auth = self.auth_type(self.conn.password)
+
+        return session
+
+    def _paginate(self, endpoint: str, payload: Optional[Dict[str, Any]] = None) -> List[Response]:
+        results = []
+        response = self.run(endpoint=endpoint, data=payload)
+        resp_json = response.json()
+        limit = resp_json["extra"]["filters"]["limit"]
+        num_total_results = resp_json["extra"]["pagination"]["total_count"]
+        num_current_results = resp_json["extra"]["pagination"]["count"]
+        results.append(response)
+
+        if not num_current_results == num_total_results:
+            _paginate_payload = payload.copy() if payload else {}
+            _paginate_payload["offset"] = limit
+
+            while True:
+                if num_current_results < num_total_results:
+                    response = self.run(endpoint=endpoint, data=_paginate_payload)
+                    resp_json = response.json()
+                    if resp_json["data"]:

Review comment:
       Certainly couldn't hurt. Perhaps this is better as a debug statement though since it's more related to the aspects of the API response rather than the task execution.




-- 
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 change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r793949682



##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,488 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+
+        return request
+
+
+class JobRunInfo(TypedDict):
+    """Type class for the ``job_run_info`` dictionary."""
+
+    account_id: int
+    run_id: int
+
+
+class DbtCloudJobRunStatus(Enum):
+    """dbt Cloud Job statuses."""
+
+    QUEUED = 1
+    STARTING = 2
+    RUNNING = 3
+    SUCCESS = 10
+    ERROR = 20
+    CANCELLED = 30
+    TERMINAL_STATUSES = (SUCCESS, ERROR, CANCELLED)
+
+    @classmethod
+    def check_is_valid(cls, statuses: Union[int, Sequence[int], Set[int]]):
+        """Validates input statuses are a known value."""
+        if isinstance(statuses, (Sequence, Set)):
+            for status in statuses:
+                cls(status)
+        else:
+            cls(statuses)
+
+    @classmethod
+    def is_terminal(cls, status: int) -> bool:
+        """Checks if the input status is that of a terminal type."""
+        cls.check_is_valid(statuses=status)
+
+        return status in cls.TERMINAL_STATUSES.value
+
+
+class DbtCloudJobRunException(AirflowException):
+    """An exception that indicates a job run failed to complete."""
+
+
+class DbtCloudHook(HttpHook):
+    """
+    Interact with dbt Cloud using the V2 API.
+
+    :param dbt_cloud_conn_id: The ID of the :ref:`dbt Cloud connection <howto/connection:dbt-cloud>`.
+    """
+
+    conn_name_attr = "dbt_cloud_conn_id"
+    default_conn_name = "dbt_cloud_default"
+    conn_type = "dbt_cloud"
+    hook_name = "dbt Cloud"
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict[str, Any]:
+        """Builds custom field behavior for the dbt Cloud connection form in the Airflow UI."""
+        return {
+            "hidden_fields": ["host", "port", "schema", "extra"],
+            "relabeling": {"login": "Account ID", "password": "API Token"},
+        }
+
+    def __init__(self, dbt_cloud_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(auth_type=TokenAuth)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.base_url = "https://cloud.getdbt.com/api/v2/accounts/"
+
+    @cached_property
+    def conn(self) -> Connection:
+        _conn = self.get_connection(self.dbt_cloud_conn_id)
+        if not _conn.password:
+            raise AirflowException("An API token is required to connect to dbt Cloud.")
+
+        return _conn
+
+    def get_conn(self, *args, **kwargs) -> Session:
+        session = Session()
+        session.auth = self.auth_type(self.conn.password)
+
+        return session
+
+    def _paginate(self, endpoint: str, payload: Optional[Dict[str, Any]] = None) -> List[Response]:
+        results = []
+        response = self.run(endpoint=endpoint, data=payload)
+        resp_json = response.json()
+        limit = resp_json["extra"]["filters"]["limit"]
+        num_total_results = resp_json["extra"]["pagination"]["total_count"]
+        num_current_results = resp_json["extra"]["pagination"]["count"]
+        results.append(response)
+
+        if not num_current_results == num_total_results:
+            _paginate_payload = payload.copy() if payload else {}
+            _paginate_payload["offset"] = limit
+
+            while True:
+                if num_current_results < num_total_results:
+                    response = self.run(endpoint=endpoint, data=_paginate_payload)
+                    resp_json = response.json()
+                    if resp_json["data"]:

Review comment:
       I removed this check. The `data` object has no bearing on pagination. Thanks for the 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] mik-laj edited a comment on pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
mik-laj edited a comment on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1028336645


   @sungchun12  DBT is used quite often with Apache Airflow, so I believe it is worth asking for reviews to give you an opportunity to share your thoughts on this contribution. The earlier the changes are introduced, the lower the cost of their implementation. 
   
   We also sometimes do not know all API features as they may not be widely promoted, but from your perspective they are important.  For example, all requests to Google API contain client info, which allows them to track API usage by a specific solution
   https://github.com/apache/airflow/blob/d353f023ff8856c00b9f054526cb2e40ff0116ae/airflow/providers/google/common/hooks/base_google.py#L340-L352
   Snowflake provider provides have a similar feature: https://github.com/apache/airflow/blob/d353f023ff8856c00b9f054526cb2e40ff0116ae/airflow/providers/snowflake/hooks/snowflake.py#L184 
   
   On the other hand, it can also be a signal for you that there is a new feature in the third project and you can promote it, e.g. by updating the documentation https://docs.getdbt.com/docs/running-a-dbt-project/running-dbt-in-production#using-airflow
   
   Does it make sense to you?


-- 
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] sungchun12 commented on pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
sungchun12 commented on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1053669113


   @josh-fell amazing work, thanks for driving this to the finish line!


-- 
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 merged pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
potiuk merged pull request #20998:
URL: https://github.com/apache/airflow/pull/20998


   


-- 
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 change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r789726196



##########
File path: airflow/providers/dbt/cloud/operators/dbt.py
##########
@@ -0,0 +1,203 @@
+# 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.
+
+import json
+from typing import TYPE_CHECKING, Any, Dict, List, Optional
+
+from airflow.models import BaseOperator, BaseOperatorLink, TaskInstance
+from airflow.providers.dbt.cloud.hooks.dbt import DbtCloudHook, DbtCloudJobRunException, DbtCloudJobRunStatus
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class DbtCloudRunJobOperatorLink(BaseOperatorLink):
+    """
+    Operator link for DbtCloudRunJobOperator. This link allows users to monitor the triggered job run
+    directly in dbt Cloud.
+    """
+
+    name = "Monitor Job Run"
+
+    def get_link(self, operator, dttm):
+        ti = TaskInstance(task=operator, execution_date=dttm)
+        job_run_url = ti.xcom_pull(task_ids=operator.task_id, key="job_run_url")
+
+        return job_run_url
+
+
+class DbtCloudRunJobOperator(BaseOperator):
+    """
+    Executes a dbt Cloud job.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:DbtCloudRunJobOperator`
+
+    :param dbt_cloud_conn_id: The connection ID for connecting to dbt Cloud.
+    :param job_id: The ID of a dbt Cloud job.
+    :param account_id: Optional. The ID of a dbt Cloud account.
+    :param trigger_reason: Optional. Description of the reason to trigger the job.
+    :param steps_override: Optional. List of dbt commands to execute when triggering the job instead of those
+        configured in dbt Cloud.
+    :param schema_override: Optional. Override the destination schema in the configured target for this job.
+    :param wait_for_termination: Flag to wait on a job run's termination.  By default, this feature is
+        enabled but could be disabled to perform an asynchronous wait for a long-running job run execution
+        using the ``DbtCloudJobRunSensor``.
+    :param timeout: Time in seconds to wait for a job run to reach a terminal status for non-asynchronous
+        waits. Used only if ``wait_for_termination`` is True. Defaults to 7 days.
+    :param check_interval: Time in seconds to check on a job run's status for non-asynchronous waits.
+        Used only if ``wait_for_termination`` is True. Defaults to 60 seconds.
+    :param additional_run_config: Optional. Any additional parameters that should be included in the API
+        request when triggering the job.
+    :return: The ID of the triggered dbt Cloud job run.
+    """
+
+    template_fields = ("dbt_cloud_conn_id", "job_id", "account_id", "trigger_reason")
+
+    operator_extra_links = (DbtCloudRunJobOperatorLink(),)
+
+    def __init__(
+        self,
+        *,
+        dbt_cloud_conn_id: str = DbtCloudHook.default_conn_name,
+        job_id: int,
+        account_id: Optional[int] = None,
+        trigger_reason: Optional[str] = None,
+        steps_override: Optional[List[str]] = None,
+        schema_override: Optional[str] = None,
+        wait_for_termination: bool = True,
+        timeout: int = 60 * 60 * 24 * 7,
+        check_interval: int = 60,
+        additional_run_config: Optional[Dict[str, Any]] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.hook = DbtCloudHook(self.dbt_cloud_conn_id)

Review comment:
       Right on. I'll pull this initialization out. 




-- 
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] subkanthi commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
subkanthi commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r790144758



##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,488 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+
+        return request
+
+
+class JobRunInfo(TypedDict):
+    """Type class for the ``job_run_info`` dictionary."""
+
+    account_id: int
+    run_id: int
+
+
+class DbtCloudJobRunStatus(Enum):
+    """dbt Cloud Job statuses."""
+
+    QUEUED = 1
+    STARTING = 2
+    RUNNING = 3
+    SUCCESS = 10
+    ERROR = 20
+    CANCELLED = 30
+    TERMINAL_STATUSES = (SUCCESS, ERROR, CANCELLED)
+
+    @classmethod
+    def check_is_valid(cls, statuses: Union[int, Sequence[int], Set[int]]):
+        """Validates input statuses are a known value."""
+        if isinstance(statuses, (Sequence, Set)):
+            for status in statuses:
+                cls(status)
+        else:
+            cls(statuses)
+
+    @classmethod
+    def is_terminal(cls, status: int) -> bool:
+        """Checks if the input status is that of a terminal type."""
+        cls.check_is_valid(statuses=status)
+
+        return status in cls.TERMINAL_STATUSES.value
+
+
+class DbtCloudJobRunException(AirflowException):
+    """An exception that indicates a job run failed to complete."""
+
+
+class DbtCloudHook(HttpHook):
+    """
+    Interact with dbt Cloud using the V2 API.
+
+    :param dbt_cloud_conn_id: The ID of the :ref:`dbt Cloud connection <howto/connection:dbt-cloud>`.
+    """
+
+    conn_name_attr = "dbt_cloud_conn_id"
+    default_conn_name = "dbt_cloud_default"
+    conn_type = "dbt_cloud"
+    hook_name = "dbt Cloud"
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict[str, Any]:
+        """Builds custom field behavior for the dbt Cloud connection form in the Airflow UI."""
+        return {
+            "hidden_fields": ["host", "port", "schema", "extra"],
+            "relabeling": {"login": "Account ID", "password": "API Token"},
+        }
+
+    def __init__(self, dbt_cloud_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(auth_type=TokenAuth)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.base_url = "https://cloud.getdbt.com/api/v2/accounts/"
+
+    @cached_property
+    def conn(self) -> Connection:
+        _conn = self.get_connection(self.dbt_cloud_conn_id)
+        if not _conn.password:
+            raise AirflowException("An API token is required to connect to dbt Cloud.")
+
+        return _conn
+
+    def get_conn(self, *args, **kwargs) -> Session:
+        session = Session()
+        session.auth = self.auth_type(self.conn.password)
+
+        return session
+
+    def _paginate(self, endpoint: str, payload: Optional[Dict[str, Any]] = None) -> List[Response]:
+        results = []
+        response = self.run(endpoint=endpoint, data=payload)
+        resp_json = response.json()
+        limit = resp_json["extra"]["filters"]["limit"]
+        num_total_results = resp_json["extra"]["pagination"]["total_count"]
+        num_current_results = resp_json["extra"]["pagination"]["count"]
+        results.append(response)
+
+        if not num_current_results == num_total_results:
+            _paginate_payload = payload.copy() if payload else {}
+            _paginate_payload["offset"] = limit
+
+            while True:
+                if num_current_results < num_total_results:
+                    response = self.run(endpoint=endpoint, data=_paginate_payload)
+                    resp_json = response.json()
+                    if resp_json["data"]:

Review comment:
       why is this check useful, it doesnt look like `resp_json['data']` is accessed below.




-- 
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 change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r789923658



##########
File path: docs/apache-airflow-providers-dbt-cloud/connections.rst
##########
@@ -0,0 +1,87 @@
+ .. 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.
+
+
+
+.. _howto/connection:dbt-cloud:
+
+Connecting to dbt Cloud
+=======================
+
+After installing the dbt Cloud provider in your Airflow environment, the corresponding connection type of
+``dbt_cloud`` will be made available. The following describes how to configure an API token and optionally
+provide an Account ID for your dbt Cloud connection.
+
+Default Connection ID
+~~~~~~~~~~~~~~~~~~~~~
+
+All hooks and operators related to dbt Cloud use ``dbt_cloud_default`` by default.
+
+
+Authenticating to the dbt Cloud API
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+To interact with the dbt Cloud API in Airflow, either a
+`User API Token <https://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/user-tokens>`__ or a
+`Service Account API Token <hhttps://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/service-tokens>`__ is
+required.
+
+
+Configuring the connection
+~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Password (required)
+    The API token to use when authenticating to the dbt Cloud API.
+
+    If using the Connection form in the Airflow UI, the token can also be stored in the "API Token" field.
+
+Login (optional)
+    The Account ID to be used as the default Account ID for dbt Cloud operators or
+    :class:`~airflow.providers.dbt.cloud.hooks.dbt.DbtCloudHook` methods. If an Account ID is provided in the
+    connection, you are not required to pass ``account_id`` to operators or hook methods. The ``account_id``
+    value will be retrieved from the Airflow connection instead. If needed, the ``account_id`` can still be
+    explicitly passed to an operator or hook method as an override the default value configured in the
+    connection.
+
+    If using the Connection form in the Airflow UI, the Account ID can also be stored in the "Account ID"
+    field.
+
+    .. note::
+
+      If an Account ID is not provided in an Airflow connection, ``account_id`` *must* be explicitly passed to
+      an operator or hook method.
+
+When specifying the connection as an environment variable, you should specify it following the standard syntax
+of a database connection. Note that all components of the URI should be URL-encoded.
+
+
+For example, to add a connection with the connection ID of "dbt_cloud_default":
+
+    When specifying an Account ID:
+
+    .. code-block:: bash
+
+        export AIRFLOW_CONN_DBT_CLOUD_DEFAULT='dbt-cloud://account_id:api_token@'
+
+    When *not* specifying an Account ID:
+
+    .. code-block:: bash
+
+        export AIRFLOW_CONN_DBT_CLOUD_DEFAULT='dbt-cloud://api_token@'
+
+More information on creating connections via environment variables can be found

Review comment:
       Oh thanks for the tip on this reference. Really nice documentation on best practices.




-- 
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] mik-laj commented on pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1028336645


   @sungchun12  DBT is used quite often with Apache Airflow, so I found it worth asking for reviews to share your thoughts on this contribution. The earlier the changes are introduced, the lower the cost of their implementation. We also sometimes do not know all API features as they may not be widely promoted, but from your perspective they are important.  For example, all requests to Google API contain client info, which allows them to track API usage by a specific solution
   https://github.com/apache/airflow/blob/d353f023ff8856c00b9f054526cb2e40ff0116ae/airflow/providers/google/common/hooks/base_google.py#L340-L352
   Snowflake provides have a similar feature: https://github.com/apache/airflow/blob/d353f023ff8856c00b9f054526cb2e40ff0116ae/airflow/providers/snowflake/hooks/snowflake.py#L184 
   
   On the other hand, it can also be a signal for you that there is new feature in the third project and you can promote it, e.g. by updating the documentation https://docs.getdbt.com/docs/running-a-dbt-project/running-dbt-in-production#using-airflow
   
   Does it make sense to you?


-- 
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 pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
potiuk commented on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1030901060


   Would you please rebase @josh-fell ?


-- 
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] mik-laj commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r789764342



##########
File path: docs/apache-airflow-providers-dbt-cloud/connections.rst
##########
@@ -0,0 +1,87 @@
+ .. 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.
+
+
+
+.. _howto/connection:dbt-cloud:
+
+Connecting to dbt Cloud
+=======================
+
+After installing the dbt Cloud provider in your Airflow environment, the corresponding connection type of
+``dbt_cloud`` will be made available. The following describes how to configure an API token and optionally
+provide an Account ID for your dbt Cloud connection.
+
+Default Connection ID
+~~~~~~~~~~~~~~~~~~~~~
+
+All hooks and operators related to dbt Cloud use ``dbt_cloud_default`` by default.
+
+
+Authenticating to the dbt Cloud API
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+To interact with the dbt Cloud API in Airflow, either a
+`User API Token <https://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/user-tokens>`__ or a
+`Service Account API Token <hhttps://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/service-tokens>`__ is
+required.
+
+
+Configuring the connection
+~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Password (required)
+    The API token to use when authenticating to the dbt Cloud API.
+
+    If using the Connection form in the Airflow UI, the token can also be stored in the "API Token" field.
+
+Login (optional)
+    The Account ID to be used as the default Account ID for dbt Cloud operators or
+    :class:`~airflow.providers.dbt.cloud.hooks.dbt.DbtCloudHook` methods. If an Account ID is provided in the
+    connection, you are not required to pass ``account_id`` to operators or hook methods. The ``account_id``
+    value will be retrieved from the Airflow connection instead. If needed, the ``account_id`` can still be
+    explicitly passed to an operator or hook method as an override the default value configured in the
+    connection.
+
+    If using the Connection form in the Airflow UI, the Account ID can also be stored in the "Account ID"
+    field.
+
+    .. note::
+
+      If an Account ID is not provided in an Airflow connection, ``account_id`` *must* be explicitly passed to
+      an operator or hook method.
+
+When specifying the connection as an environment variable, you should specify it following the standard syntax
+of a database connection. Note that all components of the URI should be URL-encoded.
+
+
+For example, to add a connection with the connection ID of "dbt_cloud_default":
+
+    When specifying an Account ID:
+
+    .. code-block:: bash
+
+        export AIRFLOW_CONN_DBT_CLOUD_DEFAULT='dbt-cloud://account_id:api_token@'
+
+    When *not* specifying an Account ID:
+
+    .. code-block:: bash
+
+        export AIRFLOW_CONN_DBT_CLOUD_DEFAULT='dbt-cloud://api_token@'

Review comment:
       ```suggestion
           export AIRFLOW_CONN_DBT_CLOUD_DEFAULT='dbt-cloud://:api_token@'
   ```




-- 
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] mik-laj commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r789237523



##########
File path: docs/apache-airflow-providers-dbt-cloud/connections.rst
##########
@@ -0,0 +1,66 @@
+ .. 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.
+
+
+
+.. _howto/connection:dbt-cloud:
+
+Connecting to dbt Cloud
+=======================
+
+After installing the dbt Cloud provider in your Airflow environment, the corresponding connection type of
+``dbt_cloud`` will be made available. The following describes how to configure an API token and optionally
+provide an Account ID for your dbt Cloud connection.
+
+Default Connection ID
+~~~~~~~~~~~~~~~~~~~~~
+
+All hooks and operators related to dbt Cloud use ``dbt_cloud_default`` by default.
+
+
+Authenticating to the dbt Cloud API
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+To interact with the dbt Cloud API in Airflow, either a
+`User API Token <https://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/user-tokens>`__ or a
+`Service Account API Token <hhttps://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/service-tokens>`__ is
+required.
+
+
+Configuring the connection
+~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Password (required)
+    The API token to use when authenticating to the dbt Cloud API.
+
+    If using the Connection form in the Airflow UI, the token can also be stored in the "API Token" field.
+
+Login (optional)
+    The Account ID to be used as the default Account ID for dbt Cloud operators or
+    :class:`~airflow.providers.dbt.cloud.hooks.dbt.DbtCloudHook` methods. If an Account ID is provided in the
+    connection, you are not required to pass ``account_id`` to operators or hook methods. The ``account_id``
+    value will be retrieved from the Airflow connection instead. If needed, the ``account_id`` can still be
+    explicitly passed to an operator or hook method as an override the default value configured in the
+    connection.
+
+    If using the Connection form in the Airflow UI, the Account ID can also be stored in the "Account ID"
+    field.
+
+    .. note::
+
+      If an Account ID is not provided in an Airflow connection, ``account_id`` *must* be explicitly passed to
+      an operator or hook method.

Review comment:
       Can you add exaample URI as bash commnd?
   For exmaple - Google URL
   ```
   export AIRFLOW_CONN_GOOGLE_CLOUD_SQL_DEFAULT='gcpcloudsql://user:XXXXXXXXX@1.1.1.1:3306/mydb?database_type=mysql&project_id=example-project&location=europe-west1&instance=testinstance&use_proxy=True&sql_proxy_use_tcp=False'
   ```
   https://airflow.apache.org/docs/apache-airflow-providers-google/stable/connections/gcp_sql.html




-- 
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] sungchun12 commented on pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
sungchun12 commented on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1029093982


   @mik-laj thanks a bunch for the follow up. I'm in conversations internally with the dbt Labs engineers to specify a User-Agent in the headers of the dbt Cloud API requests.
   
   I'll submit a pull request for the dbt Labs docs after this is merged!
   
   @josh-fell I'll follow up with you personally on the above.


-- 
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] mik-laj commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
mik-laj commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r789765200



##########
File path: docs/apache-airflow-providers-dbt-cloud/connections.rst
##########
@@ -0,0 +1,87 @@
+ .. 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.
+
+
+
+.. _howto/connection:dbt-cloud:
+
+Connecting to dbt Cloud
+=======================
+
+After installing the dbt Cloud provider in your Airflow environment, the corresponding connection type of
+``dbt_cloud`` will be made available. The following describes how to configure an API token and optionally
+provide an Account ID for your dbt Cloud connection.
+
+Default Connection ID
+~~~~~~~~~~~~~~~~~~~~~
+
+All hooks and operators related to dbt Cloud use ``dbt_cloud_default`` by default.
+
+
+Authenticating to the dbt Cloud API
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+To interact with the dbt Cloud API in Airflow, either a
+`User API Token <https://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/user-tokens>`__ or a
+`Service Account API Token <hhttps://docs.getdbt.com/docs/dbt-cloud/dbt-cloud-api/service-tokens>`__ is
+required.
+
+
+Configuring the connection
+~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+Password (required)
+    The API token to use when authenticating to the dbt Cloud API.
+
+    If using the Connection form in the Airflow UI, the token can also be stored in the "API Token" field.
+
+Login (optional)
+    The Account ID to be used as the default Account ID for dbt Cloud operators or
+    :class:`~airflow.providers.dbt.cloud.hooks.dbt.DbtCloudHook` methods. If an Account ID is provided in the
+    connection, you are not required to pass ``account_id`` to operators or hook methods. The ``account_id``
+    value will be retrieved from the Airflow connection instead. If needed, the ``account_id`` can still be
+    explicitly passed to an operator or hook method as an override the default value configured in the
+    connection.
+
+    If using the Connection form in the Airflow UI, the Account ID can also be stored in the "Account ID"
+    field.
+
+    .. note::
+
+      If an Account ID is not provided in an Airflow connection, ``account_id`` *must* be explicitly passed to
+      an operator or hook method.
+
+When specifying the connection as an environment variable, you should specify it following the standard syntax
+of a database connection. Note that all components of the URI should be URL-encoded.
+
+
+For example, to add a connection with the connection ID of "dbt_cloud_default":
+
+    When specifying an Account ID:
+
+    .. code-block:: bash
+
+        export AIRFLOW_CONN_DBT_CLOUD_DEFAULT='dbt-cloud://account_id:api_token@'
+
+    When *not* specifying an Account ID:
+
+    .. code-block:: bash
+
+        export AIRFLOW_CONN_DBT_CLOUD_DEFAULT='dbt-cloud://api_token@'
+
+More information on creating connections via environment variables can be found

Review comment:
       Please use meaningful link text. See: https://developers.google.com/style/cross-references
   
   




-- 
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] sungchun12 commented on pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
sungchun12 commented on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1028167648


   > @nathaniel-may @emmyoop @leahwicz Can you look at it?
   
   @mik-laj These people are dbt-core contributors and do not actively work on the dbt Cloud API.
   
   What did you want someone from dbt Labs to look into specifically for this pull request?


-- 
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] github-actions[bot] commented on pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1053668794


   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.


-- 
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] sungchun12 commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
sungchun12 commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r800740861



##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,487 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+

Review comment:
       Add in a user agent to the header, and dynamically pass in the version of the airflow dbt Cloud provider to track usage on the dbt Cloud side. 
   
   ```suggestion
           request.headers["User-Agent"] = f"airflow-dbt-cloud-provider-v{X.Y.Z}"
   ```




-- 
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] mik-laj commented on pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1019322566


   @nathaniel-may @emmyoop @leahwicz Can you look at it?


-- 
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 change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
josh-fell commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r790381818



##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,488 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+
+        return request
+
+
+class JobRunInfo(TypedDict):
+    """Type class for the ``job_run_info`` dictionary."""
+
+    account_id: int
+    run_id: int
+
+
+class DbtCloudJobRunStatus(Enum):
+    """dbt Cloud Job statuses."""
+
+    QUEUED = 1
+    STARTING = 2
+    RUNNING = 3
+    SUCCESS = 10
+    ERROR = 20
+    CANCELLED = 30
+    TERMINAL_STATUSES = (SUCCESS, ERROR, CANCELLED)
+
+    @classmethod
+    def check_is_valid(cls, statuses: Union[int, Sequence[int], Set[int]]):
+        """Validates input statuses are a known value."""
+        if isinstance(statuses, (Sequence, Set)):
+            for status in statuses:
+                cls(status)
+        else:
+            cls(statuses)
+
+    @classmethod
+    def is_terminal(cls, status: int) -> bool:
+        """Checks if the input status is that of a terminal type."""
+        cls.check_is_valid(statuses=status)
+
+        return status in cls.TERMINAL_STATUSES.value
+
+
+class DbtCloudJobRunException(AirflowException):
+    """An exception that indicates a job run failed to complete."""
+
+
+class DbtCloudHook(HttpHook):
+    """
+    Interact with dbt Cloud using the V2 API.
+
+    :param dbt_cloud_conn_id: The ID of the :ref:`dbt Cloud connection <howto/connection:dbt-cloud>`.
+    """
+
+    conn_name_attr = "dbt_cloud_conn_id"
+    default_conn_name = "dbt_cloud_default"
+    conn_type = "dbt_cloud"
+    hook_name = "dbt Cloud"
+
+    @staticmethod
+    def get_ui_field_behaviour() -> Dict[str, Any]:
+        """Builds custom field behavior for the dbt Cloud connection form in the Airflow UI."""
+        return {
+            "hidden_fields": ["host", "port", "schema", "extra"],
+            "relabeling": {"login": "Account ID", "password": "API Token"},
+        }
+
+    def __init__(self, dbt_cloud_conn_id: str = default_conn_name, *args, **kwargs) -> None:
+        super().__init__(auth_type=TokenAuth)
+        self.dbt_cloud_conn_id = dbt_cloud_conn_id
+        self.base_url = "https://cloud.getdbt.com/api/v2/accounts/"
+
+    @cached_property
+    def conn(self) -> Connection:
+        _conn = self.get_connection(self.dbt_cloud_conn_id)
+        if not _conn.password:
+            raise AirflowException("An API token is required to connect to dbt Cloud.")
+
+        return _conn
+
+    def get_conn(self, *args, **kwargs) -> Session:
+        session = Session()
+        session.auth = self.auth_type(self.conn.password)
+
+        return session
+
+    def _paginate(self, endpoint: str, payload: Optional[Dict[str, Any]] = None) -> List[Response]:
+        results = []
+        response = self.run(endpoint=endpoint, data=payload)
+        resp_json = response.json()
+        limit = resp_json["extra"]["filters"]["limit"]
+        num_total_results = resp_json["extra"]["pagination"]["total_count"]
+        num_current_results = resp_json["extra"]["pagination"]["count"]
+        results.append(response)
+
+        if not num_current_results == num_total_results:
+            _paginate_payload = payload.copy() if payload else {}
+            _paginate_payload["offset"] = limit
+
+            while True:
+                if num_current_results < num_total_results:
+                    response = self.run(endpoint=endpoint, data=_paginate_payload)
+                    resp_json = response.json()
+                    if resp_json["data"]:
+                        results.append(response)
+                        num_current_results += resp_json["extra"]["pagination"]["count"]
+                        _paginate_payload["offset"] += limit
+                else:
+                    break
+
+        return results
+
+    def _run_and_get_response(
+        self,
+        method: str = "GET",
+        endpoint: Optional[str] = None,
+        payload: Union[str, Dict[str, Any], None] = None,
+        paginate: bool = False,
+    ) -> Any:
+        self.method = method
+
+        if paginate:
+            if isinstance(payload, str):
+                raise ValueError("Payload cannot be a string to paginate a response.")
+
+            if endpoint:
+                return self._paginate(endpoint=endpoint, payload=payload)
+            else:
+                raise ValueError("An endpoint is needed to paginate a response.")
+
+        return self.run(endpoint=endpoint, data=payload)
+
+    def list_accounts(self) -> List[Response]:
+        """
+        Retrieves all of the dbt Cloud accounts the configured API token is authorized to access.
+
+        :return: List of request responses.
+        """
+        return self._run_and_get_response()
+
+    @fallback_to_default_account
+    def get_account(self, account_id: Optional[int] = None) -> Response:
+        """
+        Retrieves metadata for a specific dbt Cloud account.
+
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :return: The request response.
+        """
+        return self._run_and_get_response(endpoint=f"{account_id}/")
+
+    @fallback_to_default_account
+    def list_projects(self, account_id: Optional[int] = None) -> List[Response]:
+        """
+        Retrieves metadata for all projects tied to a specified dbt Cloud account.
+
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :return: List of request responses.
+        """
+        return self._run_and_get_response(endpoint=f"{account_id}/projects/", paginate=True)
+
+    @fallback_to_default_account
+    def get_project(self, project_id: int, account_id: Optional[int] = None) -> Response:
+        """
+        Retrieves metadata for a specific project.
+
+        :param project_id: The ID of a dbt Cloud project.
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :return: The request response.
+        """
+        return self._run_and_get_response(endpoint=f"{account_id}/projects/{project_id}/")
+
+    @fallback_to_default_account
+    def list_jobs(
+        self,
+        account_id: Optional[int] = None,
+        order_by: Optional[str] = None,
+        project_id: Optional[int] = 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.
+
+        :param account_id: Optional. 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: The ID of a dbt Cloud project.
+        :return: List of request responses.
+        """
+        return self._run_and_get_response(
+            endpoint=f"{account_id}/jobs/",
+            payload={"order_by": order_by, "project_id": project_id},
+            paginate=True,
+        )
+
+    @fallback_to_default_account
+    def get_job(self, job_id: int, account_id: Optional[int] = None) -> Response:
+        """
+        Retrieves metadata for a specific job.
+
+        :param job_id: The ID of a dbt Cloud job.
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :return: The request response.
+        """
+        return self._run_and_get_response(endpoint=f"{account_id}/jobs/{job_id}")
+
+    @fallback_to_default_account
+    def trigger_job_run(
+        self,
+        job_id: int,
+        cause: str,
+        account_id: Optional[int] = None,
+        steps_override: Optional[List[str]] = None,
+        schema_override: Optional[str] = None,
+        additional_run_config: Optional[Dict[str, Any]] = None,
+    ) -> Response:
+        """
+        Triggers a run of a dbt Cloud job.
+
+        :param job_id: The ID of a dbt Cloud job.
+        :param cause: Description of the reason to trigger the job.
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :param steps_override: Optional. List of dbt commands to execute when triggering the job
+            instead of those configured in dbt Cloud.
+        :param schema_override: Optional. Override the destination schema in the configured target for this
+            job.
+        :param additional_run_config: Optional. Any additional parameters that should be included in the API
+            request when triggering the job.
+        :return: The request response.
+        """
+        if additional_run_config is None:
+            additional_run_config = {}
+
+        payload = {
+            "cause": cause,
+            "steps_override": steps_override,
+            "schema_override": schema_override,
+        }
+        payload.update(additional_run_config)
+
+        return self._run_and_get_response(
+            method="POST",
+            endpoint=f"{account_id}/jobs/{job_id}/run/",
+            payload=json.dumps(payload),
+        )
+
+    @fallback_to_default_account
+    def list_job_runs(
+        self,
+        account_id: Optional[int] = None,
+        include_related: Optional[List[str]] = None,
+        job_definition_id: Optional[int] = None,
+        order_by: Optional[str] = None,
+    ) -> List[Response]:
+        """
+        Retrieves metadata for all of the dbt Cloud job runs for an account. If a ``job_definition_id`` is
+        supplied, only metadata for runs of that specific job are pulled.
+
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :param include_related: Optional. List of related fields to pull with the run.
+            Valid values are "trigger", "job", "repository", and "environment".
+        :param job_definition_id: Optional. The dbt Cloud job ID to retrieve run metadata.
+        :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``.
+        :return: List of request responses.
+        """
+        return self._run_and_get_response(
+            endpoint=f"{account_id}/runs/",
+            payload={
+                "include_related": include_related,
+                "job_definition_id": job_definition_id,
+                "order_by": order_by,
+            },
+            paginate=True,
+        )
+
+    @fallback_to_default_account
+    def get_job_run(
+        self, run_id: int, account_id: Optional[int] = None, include_related: Optional[List[str]] = None
+    ) -> Response:
+        """
+        Retrieves metadata for a specific run of a dbt Cloud job.
+
+        :param run_id: The ID of a dbt Cloud job run.
+        :param account_id: Optional. The ID of a dbt Cloud account.
+        :param include_related: Optional. List of related fields to pull with the run.
+            Valid values are "trigger", "job", "repository", and "environment".
+        :return: The request response.
+        """
+        return self._run_and_get_response(
+            endpoint=f"{account_id}/runs/{run_id}/",

Review comment:
       This should be caught by the `@fallback_to_default_account` decorator which wraps every hook method that calls a dbt Cloud API endpoint directly. The decorator checks to see if the `account_id` is not passed or if it is, if the value is `None`. If either of those conditions are met, the `account_id` set in the Airflow connection is used. In the event there is no `account_id` configured in the connection, this exception is raised: `raise AirflowException("Could not determine the dbt Cloud account.")`. Does that suffice?




-- 
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] sungchun12 commented on a change in pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
sungchun12 commented on a change in pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#discussion_r801954488



##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,487 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+

Review comment:
       Note: the committed code is different from this suggestion!

##########
File path: airflow/providers/dbt/cloud/hooks/dbt.py
##########
@@ -0,0 +1,487 @@
+# 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.
+
+import json
+import sys
+import time
+from enum import Enum
+from functools import wraps
+from inspect import signature
+from typing import Any, Callable, Dict, List, Optional, Sequence, Set, Tuple, Union
+
+from requests import PreparedRequest, Session
+from requests.auth import AuthBase
+from requests.models import Response
+
+from airflow.exceptions import AirflowException
+from airflow.models import Connection
+from airflow.providers.http.hooks.http import HttpHook
+from airflow.typing_compat import TypedDict
+
+if sys.version_info >= (3, 8):
+    from functools import cached_property
+else:
+    from cached_property import cached_property
+
+
+def fallback_to_default_account(func: Callable) -> Callable:
+    """
+    Decorator which provides a fallback value for ``account_id``. If the ``account_id`` is None or not passed
+    to the decorated function, the value will be taken from the configured dbt Cloud Airflow Connection.
+    """
+    sig = signature(func)
+
+    @wraps(func)
+    def wrapper(*args, **kwargs) -> Callable:
+        bound_args = sig.bind(*args, **kwargs)
+
+        # Check if ``account_id`` was not included in the function signature or, if it is, the value is not
+        # provided.
+        if bound_args.arguments.get("account_id") is None:
+            self = args[0]
+            default_account_id = self.conn.login
+            if not default_account_id:
+                raise AirflowException("Could not determine the dbt Cloud account.")
+
+            bound_args.arguments["account_id"] = int(default_account_id)
+
+        return func(*bound_args.args, **bound_args.kwargs)
+
+    return wrapper
+
+
+class TokenAuth(AuthBase):
+    """Helper class for Auth when executing requests."""
+
+    def __init__(self, token: str) -> None:
+        self.token = token
+
+    def __call__(self, request: PreparedRequest) -> PreparedRequest:
+        request.headers["Content-Type"] = "application/json"
+        request.headers["Authorization"] = f"Token {self.token}"
+

Review comment:
       Note: the committed code is different from this suggestion!




-- 
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] mik-laj edited a comment on pull request #20998: Add dbt Cloud provider

Posted by GitBox <gi...@apache.org>.
mik-laj edited a comment on pull request #20998:
URL: https://github.com/apache/airflow/pull/20998#issuecomment-1028336645


   @sungchun12  DBT is used quite often with Apache Airflow, so I found it worth asking for reviews to share your thoughts on this contribution. The earlier the changes are introduced, the lower the cost of their implementation. We also sometimes do not know all API features as they may not be widely promoted, but from your perspective they are important.  For example, all requests to Google API contain client info, which allows them to track API usage by a specific solution
   https://github.com/apache/airflow/blob/d353f023ff8856c00b9f054526cb2e40ff0116ae/airflow/providers/google/common/hooks/base_google.py#L340-L352
   Snowflake provider provides have a similar feature: https://github.com/apache/airflow/blob/d353f023ff8856c00b9f054526cb2e40ff0116ae/airflow/providers/snowflake/hooks/snowflake.py#L184 
   
   On the other hand, it can also be a signal for you that there is a new feature in the third project and you can promote it, e.g. by updating the documentation https://docs.getdbt.com/docs/running-a-dbt-project/running-dbt-in-production#using-airflow
   
   Does it make sense to you?


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