def poke(self, context: "Context") -> bool: hook = DbtCloudHook(self.dbt_cloud_conn_id) job_run_status = hook.get_job_run_status(run_id=self.run_id, account_id=self.account_id) if job_run_status == DbtCloudJobRunStatus.ERROR.value: raise DbtCloudJobRunException(f"Job run {self.run_id} has failed.") if job_run_status == DbtCloudJobRunStatus.CANCELLED.value: raise DbtCloudJobRunException(f"Job run {self.run_id} has been cancelled.") return job_run_status == DbtCloudJobRunStatus.SUCCESS.value
def execute(self, context: "Context") -> None: hook = DbtCloudHook(self.dbt_cloud_conn_id) response = hook.get_job_run_artifact(run_id=self.run_id, path=self.path, account_id=self.account_id, step=self.step) with open(self.output_file_name, "w") as file: if self.path.endswith(".json"): json.dump(response.json(), file) else: file.write(response.text)
def execute(self, context: "Context") -> int: if self.trigger_reason is None: self.trigger_reason = ( f"Triggered via Apache Airflow by task {self.task_id!r} in the {self.dag.dag_id} DAG." ) self.hook = DbtCloudHook(self.dbt_cloud_conn_id) trigger_job_response = self.hook.trigger_job_run( account_id=self.account_id, job_id=self.job_id, cause=self.trigger_reason, steps_override=self.steps_override, schema_override=self.schema_override, additional_run_config=self.additional_run_config, ) self.run_id = trigger_job_response.json()["data"]["id"] job_run_url = trigger_job_response.json()["data"]["href"] # Push the ``job_run_url`` value to XCom regardless of what happens during execution so that the job # run can be monitored via the operator link. context["ti"].xcom_push(key="job_run_url", value=job_run_url) if self.wait_for_termination: self.log.info("Waiting for job run %s to terminate.", str(self.run_id)) if self.hook.wait_for_job_run_status( run_id=self.run_id, account_id=self.account_id, expected_statuses=DbtCloudJobRunStatus.SUCCESS.value, check_interval=self.check_interval, timeout=self.timeout, ): self.log.info("Job run %s has completed successfully.", str(self.run_id)) else: raise DbtCloudJobRunException( f"Job run {self.run_id} has failed or has been cancelled.") return self.run_id
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.account_id = account_id self.job_id = job_id self.trigger_reason = trigger_reason self.steps_override = steps_override self.schema_override = schema_override self.wait_for_termination = wait_for_termination self.timeout = timeout self.check_interval = check_interval self.additional_run_config = additional_run_config or {} self.hook: DbtCloudHook self.run_id: int def execute(self, context: "Context") -> int: if self.trigger_reason is None: self.trigger_reason = ( f"Triggered via Apache Airflow by task {self.task_id!r} in the {self.dag.dag_id} DAG." ) self.hook = DbtCloudHook(self.dbt_cloud_conn_id) trigger_job_response = self.hook.trigger_job_run( account_id=self.account_id, job_id=self.job_id, cause=self.trigger_reason, additional_run_config=self.additional_run_config, ) self.run_id = trigger_job_response.json()["data"]["id"] job_run_url = trigger_job_response.json()["data"]["href"] # Push the ``job_run_url`` value to XCom regardless of what happens during execution so that the job # run can be monitored via the operator link. context["ti"].xcom_push(key="job_run_url", value=job_run_url) if self.wait_for_termination: self.log.info("Waiting for job run %s to terminate.", str(self.run_id)) if self.hook.wait_for_job_run_status( run_id=self.run_id, account_id=self.account_id, expected_statuses=DbtCloudJobRunStatus.SUCCESS.value, check_interval=self.check_interval, timeout=self.timeout, ): self.log.info("Job run %s has completed successfully.", str(self.run_id)) else: raise DbtCloudJobRunException( f"Job run {self.run_id} has failed or has been cancelled.") return self.run_id def on_kill(self) -> None: if self.run_id: self.hook.cancel_job_run(account_id=self.account_id, run_id=self.run_id) if self.hook.wait_for_job_run_status( run_id=self.run_id, account_id=self.account_id, expected_statuses=DbtCloudJobRunStatus.CANCELLED.value, check_interval=self.check_interval, timeout=self.timeout, ): self.log.info("Job run %s has been cancelled successfully.", str(self.run_id))