Skip to content

Introduce RUN-level OpenLineage Events for dbt Cloud Jobs in Airflow Integration #68661

@mishrasangeeta87

Description

@mishrasangeeta87

Description

Current Behavior

Airflow DAG

from datetime import datetime
from airflow.providers.dbt.cloud.operators.dbt import DbtCloudRunJobOperator

default_args = {
    "start_date": datetime(2026, 6, 9),
    "schedule_interval": None,
}

with DAG("dbt_cloud_dag", default_args=default_args) as dag:

    dbt_cloud_job = DbtCloudRunJobOperator(
        task_id="dbt_cloud_job_run",
        job_id=70471823610018,
        dbt_cloud_conn_id="dbt_cloud_default",
        wait_for_termination=True,
        check_interval=10,
        timeout=300,
    )

Airflow Logs

Getting the status of job run 70471883265346.
Current status of job run 70471883265346:  STARTING

The operator successfully triggers a dbt Cloud Job and stores the run information in Airflow XCom. The dbt Cloud run ID can be retrieved successfully.

However:

  • No OpenLineage START event is emitted for the dbt Cloud Job execution.
  • No OpenLineage COMPLETE or FAIL event is emitted for the dbt Cloud Job execution.
  • The dbt Cloud run ID (70471883265346) is not represented as an OpenLineage run.
  • The emitted model/test/source/snapshot events directly reference the Airflow DAG run through the parent facet.
  • There is no OpenLineage entity representing the dbt Cloud Job execution itself.

Current Hierarchy

 Airflow Task Run
├── dbt Model
├── dbt Model
├── dbt Test
├── dbt Snapshot
└── dbt Source

The dbt Cloud Job layer is completely missing.


Problem Statement

dbt Cloud introduces a Job abstraction that does not exist in dbt Core.

A dbt Cloud Job execution has:

  • A unique Run ID
  • Start time
  • End time
  • Status
  • Execution metadata
  • Parent orchestration context

Therefore, a dbt Cloud Job execution should be represented as a first-class OpenLineage RUN.

Currently, every model, test, source, and snapshot event emitted by dbt Cloud references the Airflow Job run as its parent. This results in an incorrect lineage hierarchy:

Airflow Task Run
├── dbt Model
├── dbt Test
├── dbt Snapshot
└── dbt Source

The actual execution hierarchy should be:

Airflow Task Run
└── dbt Cloud Job Run
    ├── dbt Model
    ├── dbt Test
    ├── dbt Snapshot
    └── dbt Source

Expected Behavior

OpenLineage should emit RUN-level events representing the dbt Cloud Job execution.

Event Flow

1. Airflow Starts dbt Cloud Job

Emit a START event for the dbt Cloud Job:

namespace=<dbt-cloud>
job=<dbt-cloud-job>
run=<dbt-cloud-run-id>
parent=<airflow-task-run>

2. dbt Resource Events

All model, source, snapshot, and test events emitted during the dbt Cloud Job execution should reference the dbt Cloud Job Run as their parent.

Expected:

parent=<dbt-cloud-run-id>

Current:

parent=<airflow-task-run-id>

3. dbt Cloud Job Completes

Emit a terminal event for the dbt Cloud Job:

namespace=<dbt-cloud>
job=<dbt-cloud-job>
run=<dbt-cloud-run-id>

or

namespace=<dbt-cloud>
job=<dbt-cloud-job>
run=<dbt-cloud-run-id>

depending on the final status.


Required Metadata

The dbt Cloud Job RUN should expose:

  • dbt Cloud Job ID
  • dbt Cloud Run ID
  • Account ID
  • Project ID (if available)
  • Environment ID (if available)
  • Start Time
  • End Time
  • Final Status
  • Parent Airflow Run Information

The dbt Cloud Run ID should become the OpenLineage run identifier for the dbt Cloud Job execution.


Benefits

  • Aligns dbt Cloud lineage with OpenLineage parent-child semantics.
  • Preserves dbt Cloud Job executions as first-class lineage entities.
  • Enables correlation using native dbt Cloud Run IDs.
  • Improves orchestration visualization.
  • Accurately represents Airflow → dbt Cloud → dbt Resource execution flow.
  • Provides consistency with other integrations such as Airflow → Spark and Airflow → Databricks.

Additional Context

This request is specific to dbt Cloud.

Unlike dbt Core, dbt Cloud introduces a Job abstraction with its own lifecycle and run identifier. The OpenLineage representation should therefore include a RUN-level entity for the dbt Cloud Job execution, positioned between the Airflow DAG run and the individual dbt resources executed within the job.

Use case/motivation

The current OpenLineage implementation for DbtCloudRunJobOperator emits lineage events for individual dbt resources (models, tests, snapshots, sources) executed within a dbt Cloud Job. However, it does not emit OpenLineage RUN-level events representing the dbt Cloud Job execution itself.

As a result, the lineage hierarchy does not follow the standard OpenLineage parent-child convention where:

  • A parent entity is represented as a RUN
  • Child executions are represented as JOBS belonging to that RUN

This makes it difficult to accurately represent dbt Cloud Job executions and their relationship with upstream orchestrators such as Airflow.

Related issues

No response

Are you willing to submit a PR?

  • Yes I am willing to submit a PR!

Code of Conduct

Metadata

Metadata

Assignees

No one assigned

    Type

    No type
    No fields configured for issues without a type.

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions