diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage.json b/providers/openlineage/tests/system/openlineage/example_openlineage.json deleted file mode 100644 index 5949941d21434..0000000000000 --- a/providers/openlineage/tests/system/openlineage/example_openlineage.json +++ /dev/null @@ -1,169 +0,0 @@ -[ - { - "eventType": "START", - "eventTime": "{{ is_datetime(result) }}", - "producer": "{{ result.startswith('https://github.com/apache/airflow/tree/providers-openlineage') }}", - "schemaURL": "{{ result.startswith('https://openlineage.io/spec') }}", - "inputs": [], - "outputs": [], - "run": { - "runId": "{{ is_uuid(result) }}", - "facets": { - "parent": { - "job": { - "namespace": "{{ result is string }}", - "name": "openlineage_basic_dag" - }, - "run": { - "runId": "{{ is_uuid(result) }}" - } - }, - "airflow": { - "dag": { - "dag_id": "openlineage_basic_dag", - "fileloc": "{{ result.endswith('openlineage/example_openlineage.py') }}", - "owner": "airflow", - "start_date": "{{ is_datetime(result) }}" - }, - "dagRun": { - "conf": {}, - "dag_id": "openlineage_basic_dag", - "data_interval_end": "{{ is_datetime(result) }}", - "data_interval_start": "{{ is_datetime(result) }}", - "start_date": "{{ is_datetime(result) }}" - }, - "taskInstance": { - "try_number": "{{ result is number }}" - }, - "task": { - "inlets": "[]", - "mapped": false, - "outlets": "[]", - "task_id": "do_nothing_task", - "trigger_rule": "all_success", - "operator_class": "PythonOperator", - "retries": "{{ result is number }}", - "depends_on_past": false, - "executor_config": {}, - "priority_weight": 1, - "multiple_outputs": false, - "upstream_task_ids": "[]", - "downstream_task_ids": "['check_events']", - "operator_class_path": "{{ result.endswith('.PythonOperator') }}", - "wait_for_downstream": false, - "retry_exponential_backoff": false, - "wait_for_past_depends_before_skipping": false - }, - "taskUuid": "{{ is_uuid(result) }}" - }, - "nominalTime": { - "nominalEndTime": "{{ is_datetime(result) }}", - "nominalStartTime": "{{ is_datetime(result) }}" - }, - "processing_engine": { - "name": "Airflow", - "openlineageAdapterVersion": "{{ result is string }}", - "version": "{{ result is string }}" - } - } - }, - "job": { - "namespace": "{{ result is string }}", - "name": "openlineage_basic_dag.do_nothing_task", - "facets": { - "jobType": { - "integration": "AIRFLOW", - "jobType": "TASK", - "processingType": "BATCH" - }, - "ownership": { - "owners": [ - { - "name": "{{ result is string }}" - } - ] - }, - "sourceCode": { - "language": "python", - "sourceCode": "def do_nothing():\n pass\n" - } - } - } - }, - { - "eventType": "COMPLETE", - "eventTime": "{{ is_datetime(result) }}", - "run": { - "runId": "{{ is_uuid(result) }}", - "facets": { - "parent": { - "job": { - "namespace": "{{ result is string }}", - "name": "openlineage_basic_dag" - }, - "run": { - "runId": "{{ is_uuid(result) }}" - } - }, - "airflow": { - "dag": { - "dag_id": "openlineage_basic_dag", - "fileloc": "{{ result.endswith('openlineage/example_openlineage.py') }}", - "owner": "airflow", - "start_date": "{{ is_datetime(result) }}" - }, - "dagRun": { - "conf": {}, - "dag_id": "openlineage_basic_dag", - "data_interval_end": "{{ is_datetime(result) }}", - "data_interval_start": "{{ is_datetime(result) }}", - "start_date": "{{ is_datetime(result) }}" - }, - "taskInstance": { - "try_number": "{{ result is number }}" - }, - "task": { - "inlets": "[]", - "mapped": false, - "outlets": "[]", - "task_id": "do_nothing_task", - "trigger_rule": "all_success", - "operator_class": "PythonOperator", - "retries": "{{ result is number }}", - "depends_on_past": false, - "executor_config": {}, - "priority_weight": 1, - "multiple_outputs": false, - "upstream_task_ids": "[]", - "downstream_task_ids": "['check_events']", - "operator_class_path": "{{ result.endswith('.PythonOperator') }}", - "wait_for_downstream": false, - "retry_exponential_backoff": false, - "wait_for_past_depends_before_skipping": false - }, - "taskUuid": "{{ is_uuid(result) }}" - }, - "processing_engine": { - "name": "Airflow", - "openlineageAdapterVersion": "{{ result is string }}", - "version": "{{ result is string }}" - } - } - }, - "job": { - "namespace": "{{ result is string }}", - "name": "openlineage_basic_dag.do_nothing_task", - "facets": { - "jobType": { - "integration": "AIRFLOW", - "jobType": "TASK", - "processingType": "BATCH" - }, - "sourceCode": { - "language": "python", - "sourceCode": "def do_nothing():\n pass\n" - } - } - } - } -] diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_base_complex_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_base_complex_dag.py new file mode 100644 index 0000000000000..a8cadc73fef72 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_base_complex_dag.py @@ -0,0 +1,132 @@ +# 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. +""" +Complex DAG without schedule, with multiple operators, task groups, dependencies etc. + +It checks: + - required keys + - field formats and types + - number of task events (one start, one complete) + - if EmptyOperator will emit OL events with callback or outlet + - if EmptyOperator without modification will not emit OL events + - if CustomOperator without Extractor will emit OL events + - task groups serialization without dependencies + - additional task configuration attrs (owner, max_active_tis_per_dag etc.) +""" + +from __future__ import annotations + +from datetime import datetime, timedelta + +from airflow import DAG +from airflow.models import Variable +from airflow.providers.common.compat.assets import Asset +from airflow.providers.standard.operators.bash import BashOperator +from airflow.providers.standard.operators.empty import EmptyOperator +from airflow.providers.standard.operators.python import PythonOperator +from airflow.utils.task_group import TaskGroup + +from system.openlineage.expected_events import AIRFLOW_VERSION, get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + + +def check_events_number_func(): + for event_type in ("start", "complete", "fail"): + try: + Variable.get(key=f"openlineage_base_complex_dag.task_0.event.{event_type}", deserialize_json=True) + except Exception: + pass + else: + raise ValueError("Expected no events for task `task_0`.") + + +def do_nothing(): + pass + + +class SomeCustomOperator(BashOperator): + def __init__(self, **kwargs): + # Just to test that these attrs are included in OL event + self.deferrable = True + self.external_dag_id = "external_dag_id" + self.external_task_id = "external_task_id" + super().__init__(**kwargs) + + +DAG_ID = "openlineage_base_complex_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=None, + catchup=False, + description="OpenLineage complex DAG description", + owner_links={"airflow": "https://airflow.apache.org/"}, + tags=["first", "second@", "with'quote"], + default_args={"retries": 0}, +) as dag: + # task_0 will not emit any events, but the owner will be picked up and added to DAG + task_0 = EmptyOperator(task_id="task_0", owner="owner1") + task_1 = BashOperator( + task_id="task_1.id.with.dots", + bash_command="exit 0;", + owner="owner2", + execution_timeout=timedelta(seconds=456), + ) + task_2 = PythonOperator( + task_id="task_2", + python_callable=do_nothing, + inlets=[Asset(uri="s3://bucket2/dir2/file2.txt")], + max_retry_delay=42, + ) + task_3 = EmptyOperator(task_id="task_3", outlets=[Asset(uri="s3://bucket/dir/file.txt")]) + task_4 = SomeCustomOperator( + task_id="task_4", + bash_command="exit 0;", + owner="owner3", + max_active_tis_per_dag=7, + max_active_tis_per_dagrun=2, + ) + + with TaskGroup("section_1", prefix_group_id=True) as tg: + task_5 = PythonOperator(task_id="task_5", python_callable=lambda: 1) + with TaskGroup("section_2", parent_group=tg, tooltip="group_tooltip") as tg2: + if AIRFLOW_VERSION.major == 3: + add_args = {"run_as_user": "some_user"} # Random user break task execution on AF2 + else: + add_args = {"sla": timedelta(seconds=123)} # type: ignore[dict-item] # SLA is not present in AF3 yet + task_6 = EmptyOperator(task_id="task_6", on_success_callback=lambda x: print(1), **add_args) + with TaskGroup("section_3", parent_group=tg2): + task_7 = PythonOperator(task_id="task_7", python_callable=lambda: 1) + + check_events_number = PythonOperator( + task_id="check_events_number", python_callable=check_events_number_func + ) + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + task_1 >> [task_2, task_7] >> check_events_number + task_2 >> task_3 >> [task_4, task_5] >> task_6 >> check_events_number + check_events_number >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_base_simple_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_base_simple_dag.py new file mode 100644 index 0000000000000..7310b51cf167b --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_base_simple_dag.py @@ -0,0 +1,62 @@ +# 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. +""" +Simple DAG without schedule and extra args, with one operator, to verify OpenLineage event integrity. + +It checks: + - required keys + - field formats and types + - number of task events (one start, one complete) +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.standard.operators.python import PythonOperator + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + + +def do_nothing(): + pass + + +DAG_ID = "openlineage_base_simple_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=None, + catchup=False, + default_args={"retries": 0}, +) as dag: + do_nothing_task = PythonOperator(task_id="do_nothing_task", python_callable=do_nothing) + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + do_nothing_task >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_defer.json b/providers/openlineage/tests/system/openlineage/example_openlineage_defer.json deleted file mode 100644 index 05c64292c2a01..0000000000000 --- a/providers/openlineage/tests/system/openlineage/example_openlineage_defer.json +++ /dev/null @@ -1,38 +0,0 @@ -[ - { - "eventType": "START", - "eventTime": "{{ is_datetime(result) }}", - "run": { - "runId": "{{ is_uuid(result) }}" - }, - "job": { - "namespace": "{{ result is string }}", - "name": "openlineage_basic_defer_dag.wait", - "facets": { - "jobType": { - "integration": "AIRFLOW", - "jobType": "TASK", - "processingType": "BATCH" - } - } - } - }, - { - "eventType": "COMPLETE", - "eventTime": "{{ is_datetime(result) }}", - "run": { - "runId": "{{ is_uuid(result) }}" - }, - "job": { - "namespace": "{{ result is string }}", - "name": "openlineage_basic_defer_dag.wait", - "facets": { - "jobType": { - "integration": "AIRFLOW", - "jobType": "TASK", - "processingType": "BATCH" - } - } - } - } -] diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_defer.py b/providers/openlineage/tests/system/openlineage/example_openlineage_defer_simple_dag.py similarity index 55% rename from providers/openlineage/tests/system/openlineage/example_openlineage_defer.py rename to providers/openlineage/tests/system/openlineage/example_openlineage_defer_simple_dag.py index 17410366c9274..14e04b9c961a9 100644 --- a/providers/openlineage/tests/system/openlineage/example_openlineage_defer.py +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_defer_simple_dag.py @@ -14,13 +14,18 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +""" +Simple DAG with deferrable operator. + +It checks: + - that at least two task START events (before and after deferral) are emitted and + the try_num remains at 1 +""" + from __future__ import annotations -import os +import warnings from datetime import datetime, timedelta -from pathlib import Path - -import pytest from airflow import DAG from airflow.exceptions import AirflowProviderDeprecationWarning @@ -28,42 +33,42 @@ from airflow.providers.standard.operators.python import PythonOperator from airflow.providers.standard.sensors.time_delta import TimeDeltaSensorAsync +from system.openlineage.expected_events import get_expected_event_file_path from system.openlineage.operator import OpenLineageTestOperator -def my_task(task_number): - print(os.getcwd()) - print(f"Executing task number: {task_number}") - - -def check_start_amount_func(): - events = Variable.get(key="openlineage_basic_defer_dag.wait.event.start", deserialize_json=True) +def check_events_number_func(): + events = Variable.get(key="openlineage_defer_simple_dag.wait.event.start", deserialize_json=True) if len(events) < 2: - raise ValueError(f"Expected at least 2 events, got {len(events)}") + raise ValueError(f"Expected at least 2 START events for task `wait`, got {len(events)}") + +DAG_ID = "openlineage_defer_simple_dag" with DAG( - dag_id="openlineage_basic_defer_dag", + dag_id=DAG_ID, start_date=datetime(2021, 1, 1), schedule=None, catchup=False, + default_args={"retries": 0}, ) as dag: - with pytest.warns(AirflowProviderDeprecationWarning): - # Timedelta is compared to the DAGRun start timestamp, which can occur long before a worker picks up the - # task. We need to ensure the sensor gets deferred at least once, so setting 180s. + # Timedelta is compared to the DAGRun start timestamp, which can occur long before a worker picks up the + # task. We need to ensure the sensor gets deferred at least once, so setting 180s. + with warnings.catch_warnings(): # TODO Switch to TimeDeltaSensor when deferrable is released + warnings.simplefilter("ignore", AirflowProviderDeprecationWarning) wait = TimeDeltaSensorAsync(task_id="wait", delta=timedelta(seconds=180)) - check_start_events_amount = PythonOperator( - task_id="check_start_events_amount", python_callable=check_start_amount_func - ) + check_events_number = PythonOperator( + task_id="check_events_number", python_callable=check_events_number_func + ) - check_events = OpenLineageTestOperator( - task_id="check_events", - file_path=str(Path(__file__).parent / "example_openlineage_defer.json"), - allow_duplicate_events=True, - ) + check_events = OpenLineageTestOperator( + task_id="check_events", + file_path=get_expected_event_file_path(DAG_ID), + allow_duplicate_events=True, + ) - wait >> check_start_events_amount >> check_events + wait >> check_events_number >> check_events from tests_common.test_utils.system_tests import get_test_run # noqa: E402 diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_mapped_simple_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_mapped_simple_dag.py new file mode 100644 index 0000000000000..7b812264c260b --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_mapped_simple_dag.py @@ -0,0 +1,85 @@ +# 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. +""" +Simple DAG with mapped task. + +It checks: + - task's `mapped` attribute + - taskInstance's `map_index` attribute + - number of OL events emitted for mapped task +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.decorators import task +from airflow.models import Variable +from airflow.providers.standard.operators.python import PythonOperator + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + + +def check_events_number_func(): + for event_type in ("start", "complete"): + events = Variable.get( + key=f"openlineage_mapped_simple_dag.add_one.event.{event_type}", deserialize_json=True + ) + if len(events) != 2: + raise ValueError( + f"Expected exactly 2 {event_type.upper()} events for task `add_one`, got {len(events)}" + ) + + +DAG_ID = "openlineage_mapped_simple_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=None, + catchup=False, + default_args={"retries": 0}, +) as dag: + + @task(max_active_tis_per_dagrun=1) # Execute sequentially to not overwrite each other OL events + def add_one(x: int): + return x + 1 + + @task + def sum_it(values): + total = sum(values) + print(f"Total was {total}") + + added_values = add_one.expand(x=[1, 2]) + + check_events_number = PythonOperator( + task_id="check_events_number", python_callable=check_events_number_func + ) + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID), allow_duplicate_events=True + ) + + sum_it(added_values) >> check_events_number >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_asset_or_time_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_asset_or_time_dag.py new file mode 100644 index 0000000000000..f4944cc1b39dd --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_asset_or_time_dag.py @@ -0,0 +1,78 @@ +# 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. +""" +Simple DAG with asset or time schedule. + +It checks: + - schedule serialization +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.common.compat.assets import Asset +from airflow.providers.standard.operators.bash import BashOperator +from airflow.timetables.trigger import CronTriggerTimetable + +from system.openlineage.expected_events import AIRFLOW_VERSION, get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +DAG_ID = "openlineage_schedule_asset_or_time_dag" + +if AIRFLOW_VERSION.major == 3: + from airflow.timetables.assets import AssetOrTimeSchedule + + schedule = AssetOrTimeSchedule( + timetable=CronTriggerTimetable("21 13 29 2 4", timezone="UTC"), + assets=( + (Asset(uri="s3://bucket/file.txt", extra={"a": 1}) | Asset(uri="s3://bucket2/file.txt")) + & (Asset(uri="s3://bucket3/file.txt") | Asset(uri="s3://bucket4/file.txt", extra={"b": 2})) + ), + ) +else: + from airflow.timetables.datasets import DatasetOrTimeSchedule + + schedule = DatasetOrTimeSchedule( # type: ignore[assignment] # re-defining schedule, but it's for different AF + timetable=CronTriggerTimetable("21 13 29 2 4", timezone="UTC"), + datasets=( + (Asset(uri="s3://bucket/file.txt", extra={"a": 1}) | Asset(uri="s3://bucket2/file.txt")) + & (Asset(uri="s3://bucket3/file.txt") | Asset(uri="s3://bucket4/file.txt", extra={"b": 2})) + ), + ) + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2025, 5, 1), + schedule=schedule, + catchup=False, + default_args={"retries": 0}, +) as dag: + do_nothing_task = BashOperator(task_id="do_nothing_task", bash_command="sleep 10;") + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + do_nothing_task >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_cron_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_cron_dag.py new file mode 100644 index 0000000000000..77750f760c397 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_cron_dag.py @@ -0,0 +1,55 @@ +# 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. +""" +Simple DAG with cron schedule. + +It checks: + - schedule serialization +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.standard.operators.bash import BashOperator + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +DAG_ID = "openlineage_schedule_cron_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule="21 13 29 2 4", # Unlikely to ever get triggered by itself, February 29th and Thursday + catchup=False, + default_args={"retries": 0}, +) as dag: + do_nothing_task = BashOperator(task_id="do_nothing_task", bash_command="sleep 10;") + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + do_nothing_task >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_list_complex_assets_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_list_complex_assets_dag.py new file mode 100644 index 0000000000000..c3e38ee235879 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_list_complex_assets_dag.py @@ -0,0 +1,67 @@ +# 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. +""" +Simple DAG with multiple asset logical condition in list schedule for Airflow 3. +Use of list will result in the whole condition being wrapped with additional `asset_all`. + +It checks: + - schedule serialization +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.common.compat.assets import Asset +from airflow.providers.standard.operators.bash import BashOperator + +from system.openlineage.expected_events import AIRFLOW_VERSION, get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +if AIRFLOW_VERSION.major == 3: + schedule = [ + (Asset(uri="s3://bucket/file.txt", extra={"a": 1}) | Asset(uri="s3://bucket2/file.txt")) + & (Asset(uri="s3://bucket3/file.txt") | Asset(uri="s3://bucket4/file.txt", extra={"b": 2})) + ] +else: + # Logical Asset condition wrapped in list breaks DAG processing in Airflow 2 - check is skipped + schedule = None # type: ignore[assignment] + + +DAG_ID = "openlineage_schedule_list_complex_assets_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=schedule, + catchup=False, + default_args={"retries": 0}, +) as dag: + do_nothing_task = BashOperator(task_id="do_nothing_task", bash_command="sleep 10;") + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + do_nothing_task >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_list_multiple_assets_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_list_multiple_assets_dag.py new file mode 100644 index 0000000000000..8997ac02dfa7a --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_list_multiple_assets_dag.py @@ -0,0 +1,56 @@ +# 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. +""" +Simple DAG with multiple asset in list schedule. + +It checks: + - schedule serialization +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.common.compat.assets import Asset +from airflow.providers.standard.operators.bash import BashOperator + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +DAG_ID = "openlineage_schedule_list_multiple_assets_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=[Asset(uri="s3://bucket/file.txt", extra={"a": 1}), Asset(uri="s3://bucket2/file.txt")], + catchup=False, + default_args={"retries": 0}, +) as dag: + do_nothing_task = BashOperator(task_id="do_nothing_task", bash_command="sleep 10;") + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + do_nothing_task >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_list_single_asset_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_list_single_asset_dag.py new file mode 100644 index 0000000000000..db0aa95527e48 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_list_single_asset_dag.py @@ -0,0 +1,56 @@ +# 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. +""" +Simple DAG with single asset in list schedule. + +It checks: + - schedule serialization +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.common.compat.assets import Asset +from airflow.providers.standard.operators.bash import BashOperator + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +DAG_ID = "openlineage_schedule_list_single_asset_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=[Asset(uri="s3://bucket/file.txt", extra={"some_extra": 123})], + catchup=False, + default_args={"retries": 0}, +) as dag: + do_nothing_task = BashOperator(task_id="do_nothing_task", bash_command="sleep 10;") + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + do_nothing_task >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_multiple_assets_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_multiple_assets_dag.py new file mode 100644 index 0000000000000..a8228ca09f1da --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_multiple_assets_dag.py @@ -0,0 +1,59 @@ +# 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. +""" +Simple DAG with multiple assets with logical conditions in schedule. + +It checks: + - schedule serialization +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.common.compat.assets import Asset +from airflow.providers.standard.operators.bash import BashOperator + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +DAG_ID = "openlineage_schedule_multiple_assets_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=( + (Asset(uri="s3://bucket/file.txt", extra={"a": 1}) | Asset(uri="s3://bucket2/file.txt")) + & (Asset(uri="s3://bucket3/file.txt") | Asset(uri="s3://bucket4/file.txt", extra={"b": 2})) + ), + catchup=False, + default_args={"retries": 0}, +) as dag: + do_nothing_task = BashOperator(task_id="do_nothing_task", bash_command="sleep 10;") + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + do_nothing_task >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_single_asset_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_single_asset_dag.py new file mode 100644 index 0000000000000..40b0a1d6cbf9e --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_single_asset_dag.py @@ -0,0 +1,56 @@ +# 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. +""" +Simple DAG with single asset direct schedule. + +It checks: + - schedule serialization +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.common.compat.assets import Asset +from airflow.providers.standard.operators.bash import BashOperator + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +DAG_ID = "openlineage_schedule_single_asset_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=Asset(uri="s3://bucket/file.txt", extra={"some_extra": 123}), + catchup=False, + default_args={"retries": 0}, +) as dag: + do_nothing_task = BashOperator(task_id="do_nothing_task", bash_command="sleep 10;") + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + do_nothing_task >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_timetable_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_timetable_dag.py new file mode 100644 index 0000000000000..7e552374e2d21 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_schedule_timetable_dag.py @@ -0,0 +1,65 @@ +# 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. +""" +Simple DAG with timetable schedule. + +It checks: + - schedule serialization +""" + +from __future__ import annotations + +from datetime import datetime + +import pendulum + +from airflow import DAG +from airflow.providers.standard.operators.bash import BashOperator +from airflow.timetables.events import EventsTimetable + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +DAG_ID = "openlineage_schedule_timetable_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=EventsTimetable( # Unlikely to ever get triggered by itself + event_dates=[ + pendulum.datetime(2095, 3, 3, 8, 27, tz="America/Chicago"), + pendulum.datetime(2095, 3, 17, 8, 27, tz="America/Chicago"), + pendulum.datetime(2095, 3, 22, 20, 50, tz="America/Chicago"), + ], + description="My Team's Baseball Games", + ), + catchup=False, + default_args={"retries": 0}, +) as dag: + do_nothing_task = BashOperator(task_id="do_nothing_task", bash_command="sleep 10;") + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + do_nothing_task >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_task_groups_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_task_groups_dag.py new file mode 100644 index 0000000000000..9c4e5b9c4732b --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_task_groups_dag.py @@ -0,0 +1,62 @@ +# 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. +""" +Simple DAG with task group dependency + +It checks: + - task's `task_group` attr +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.standard.operators.python import PythonOperator +from airflow.utils.task_group import TaskGroup + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +DAG_ID = "openlineage_task_groups_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=None, + catchup=False, + default_args={"retries": 0}, +) as dag: + task_0 = PythonOperator(task_id="task_0", python_callable=lambda: print("task 0")) + + with TaskGroup("tg1") as tg1: + task_1 = PythonOperator(task_id="task_1", python_callable=lambda: print("task 1")) + + with TaskGroup("tg2") as tg2: + task_2 = PythonOperator(task_id="task_2", python_callable=lambda: print("task 2")) + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + task_0 >> tg1 >> tg2 >> check_events + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage_trigger_dag.py b/providers/openlineage/tests/system/openlineage/example_openlineage_trigger_dag.py new file mode 100644 index 0000000000000..4eac16070628d --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_trigger_dag.py @@ -0,0 +1,74 @@ +# 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. +""" +Simple DAG that triggers another simple DAG. + +It checks: + - task's trigger_dag_id + - DAGRun START and COMPLETE events, for the triggered DAG +""" + +from __future__ import annotations + +from datetime import datetime + +from airflow import DAG +from airflow.providers.standard.operators.bash import BashOperator +from airflow.providers.standard.operators.trigger_dagrun import TriggerDagRunOperator + +from system.openlineage.expected_events import get_expected_event_file_path +from system.openlineage.operator import OpenLineageTestOperator + +DAG_ID = "openlineage_trigger_dag" + +with DAG( + dag_id=DAG_ID, + start_date=datetime(2021, 1, 1), + schedule=None, + catchup=False, + default_args={"retries": 0}, +) as dag: + trigger_dagrun = TriggerDagRunOperator( + task_id="trigger_dagrun", + trigger_dag_id="openlineage_trigger_dag_child__notrigger", + trigger_run_id=f"openlineage_trigger_dag_triggering_child_{datetime.now().isoformat()}", + wait_for_completion=True, + conf={"some_config": "value1"}, + poke_interval=10, + ) + + check_events = OpenLineageTestOperator( + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) + ) + + trigger_dagrun >> check_events + + +with DAG( + dag_id="openlineage_trigger_dag_child__notrigger", + start_date=datetime(2021, 1, 1), + schedule=None, + catchup=False, + default_args={"retries": 0}, +) as child_dag: + do_nothing_task = BashOperator(task_id="do_nothing_task", bash_command="sleep 10;") + + +from tests_common.test_utils.system_tests import get_test_run # noqa: E402 + +# Needed to run the example DAG with pytest (see: tests/system/README.md#run_via_pytest) +test_run = get_test_run(dag) diff --git a/providers/openlineage/tests/system/openlineage/example_openlineage.py b/providers/openlineage/tests/system/openlineage/example_openlineage_versioned_dag.py similarity index 70% rename from providers/openlineage/tests/system/openlineage/example_openlineage.py rename to providers/openlineage/tests/system/openlineage/example_openlineage_versioned_dag.py index f823898f2544a..a36c65d7e717d 100644 --- a/providers/openlineage/tests/system/openlineage/example_openlineage.py +++ b/providers/openlineage/tests/system/openlineage/example_openlineage_versioned_dag.py @@ -14,14 +14,23 @@ # KIND, either express or implied. See the License for the # specific language governing permissions and limitations # under the License. +""" +Simple DAG to verify version attributes for AF3. +This DAG should only be run when versioned DagBundle is present. + +It checks: + - DagRun version related attributes + - TaskInstance version related attributes +""" + from __future__ import annotations from datetime import datetime -from pathlib import Path from airflow import DAG from airflow.providers.standard.operators.python import PythonOperator +from system.openlineage.expected_events import get_expected_event_file_path from system.openlineage.operator import OpenLineageTestOperator @@ -29,21 +38,22 @@ def do_nothing(): pass -# Instantiate the DAG +DAG_ID = "openlineage_versioned_dag" + with DAG( - "openlineage_basic_dag", + dag_id=DAG_ID, start_date=datetime(2021, 1, 1), schedule=None, catchup=False, + default_args={"retries": 0}, ) as dag: - nothing_task = PythonOperator(task_id="do_nothing_task", python_callable=do_nothing) + do_nothing_task = PythonOperator(task_id="do_nothing_task", python_callable=do_nothing) check_events = OpenLineageTestOperator( - task_id="check_events", - file_path=str(Path(__file__).parent / "example_openlineage.json"), + task_id="check_events", file_path=get_expected_event_file_path(DAG_ID) ) - nothing_task >> check_events + do_nothing_task >> check_events from tests_common.test_utils.system_tests import get_test_run # noqa: E402 diff --git a/providers/openlineage/tests/system/openlineage/expected_events/__init__.py b/providers/openlineage/tests/system/openlineage/expected_events/__init__.py new file mode 100644 index 0000000000000..6b65722cb5372 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/__init__.py @@ -0,0 +1,75 @@ +# 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. +from __future__ import annotations + +import os +from pathlib import Path + +from packaging.version import Version + +from airflow import __version__ + +AIRFLOW_VERSION = Version(__version__) + + +def get_expected_event_file_path(dag_id: str) -> str: + """ + Retrieve the file path to the OpenLineage expected events JSON file for a given DAG ID, + taking into account the Airflow version currently in use. + + Since expected event outputs may differ between Airflow versions, this function attempts + to locate the most specific expected event file available for the running Airflow version, + following this precedence order: + + 1. A file named `{dag_id}__af{major_version}_{minor_version}.json` + (e.g., `example_dag__af2_10.json` for Airflow 2.10.x) + 2. A file named `{dag_id}__af{major_version}.json` + (e.g., `example_dag__af3.json` for any Airflow 3.x version) + 3. A generic file named `{dag_id}.json` without version suffix + (e.g., `example_dag.json` for any Airflow version) + + The function returns the path to the first existing file found in this order. + We expect all the files to follow the naming convention described above. + + If none of the expected files exist, it raises a ValueError indicating that no matching + expected event file could be found. + + Args: + dag_id: The identifier of the DAG whose expected event file is to be retrieved. + + Returns: + The file path to the appropriate expected events JSON file. + + Raises: + ValueError: If no expected event files matching the DAG ID and Airflow version are found. + """ + base_path = Path(__file__).parent + + paths_to_check = ( + str(base_path / f"{dag_id}__af{AIRFLOW_VERSION.major}_{AIRFLOW_VERSION.minor}.json"), + str(base_path / f"{dag_id}__af{AIRFLOW_VERSION.major}.json"), + str(base_path / f"{dag_id}.json"), + ) + + for path in paths_to_check: + if os.path.exists(path): + return path + + raise ValueError( + f"Could not locate expected event files for dag_id {dag_id}. " + f"Expected one of the following files: `{paths_to_check}`" + ) diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_complex_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_complex_dag__af2.json new file mode 100644 index 0000000000000..3fb89997bd119 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_complex_dag__af2.json @@ -0,0 +1,1436 @@ +[ + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "airflowDagRun": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "DAG", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "airflow": { + "taskGroups": { + "section_1": { + "ui_color": "CornflowerBlue", + "ui_fgcolor": "#000", + "ui_label": "section_1" + }, + "section_1.section_2": { + "parent_group": "section_1", + "ui_color": "CornflowerBlue", + "ui_fgcolor": "#000", + "ui_label": "section_2" + }, + "section_1.section_2.section_3": { + "parent_group": "section_1.section_2", + "ui_color": "CornflowerBlue", + "ui_fgcolor": "#000", + "ui_label": "section_3" + } + }, + "taskTree": {}, + "tasks": { + "section_1.section_2.section_3.task_7": { + "downstream_task_ids": [ + "check_events_number" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.python.PythonOperator", + "task_group": "section_1.section_2.section_3", + "ui_color": "#ffefeb", + "ui_fgcolor": "#000", + "ui_label": "task_7" + }, + "section_1.section_2.task_6": { + "downstream_task_ids": [ + "check_events_number" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.empty.EmptyOperator", + "task_group": "section_1.section_2", + "ui_color": "#e8f7e4", + "ui_fgcolor": "#000", + "ui_label": "task_6" + }, + "section_1.task_5": { + "downstream_task_ids": [ + "section_1.section_2.task_6" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.python.PythonOperator", + "task_group": "section_1", + "ui_color": "#ffefeb", + "ui_fgcolor": "#000", + "ui_label": "task_5" + }, + "task_0": { + "downstream_task_ids": [], + "emits_ol_events": false, + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.empty.EmptyOperator", + "ui_color": "#e8f7e4", + "ui_fgcolor": "#000", + "ui_label": "task_0" + }, + "task_1.id.with.dots": { + "downstream_task_ids": "{{ result | sort == ['section_1.section_2.section_3.task_7', 'task_2'] }}", + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.bash.BashOperator", + "ui_color": "#f0ede4", + "ui_fgcolor": "#000", + "ui_label": "task_1.id.with.dots" + }, + "task_2": { + "downstream_task_ids": "{{ result | sort == ['check_events_number', 'task_3'] }}", + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.python.PythonOperator", + "ui_color": "#ffefeb", + "ui_fgcolor": "#000", + "ui_label": "task_2" + }, + "task_3": { + "downstream_task_ids": "{{ result | sort == ['section_1.task_5', 'task_4'] }}", + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.empty.EmptyOperator", + "ui_color": "#e8f7e4", + "ui_fgcolor": "#000", + "ui_label": "task_3" + }, + "task_4": { + "downstream_task_ids": [ + "section_1.section_2.task_6" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "{{ result.endswith('openlineage_base_complex_dag.SomeCustomOperator') }}", + "ui_color": "#f0ede4", + "ui_fgcolor": "#000", + "ui_label": "task_4" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/JobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=task_1.id.with.dots.*\") }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_2' in result and 'section_1.section_2.section_3.task_7' in result }}", + "execution_timeout": "456.0 seconds", + "executor_config": {}, + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "multiple_outputs": false, + "operator_class": "BashOperator", + "operator_class_path": "airflow.providers.standard.operators.bash.BashOperator", + "outlets": "[]", + "owner": "owner2", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_1.id.with.dots", + "trigger_rule": "all_success", + "upstream_task_ids": "[]", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_1.id.with.dots", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "sourceCode": { + "language": "bash", + "sourceCode": "exit 0;", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/SourceCodeJobFacet.json\\#\\/\\$defs\\/SourceCodeJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "duration": "{{ result is number }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=task_1.id.with.dots.*\") }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_2' in result and 'section_1.section_2.section_3.task_7' in result }}", + "execution_timeout": "456.0 seconds", + "executor_config": {}, + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "multiple_outputs": false, + "operator_class": "BashOperator", + "operator_class_path": "airflow.providers.standard.operators.bash.BashOperator", + "outlets": "[]", + "owner": "owner2", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_1.id.with.dots", + "trigger_rule": "all_success", + "upstream_task_ids": "[]", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_1.id.with.dots", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=task_2.*\") }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_3' in result and 'check_events_number' in result }}", + "executor_config": {}, + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[{'uri': 's3://bucket2/dir2/file2.txt', 'extra': None}]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "max_retry_delay": "42.0 seconds", + "multiple_outputs": false, + "operator_class": "PythonOperator", + "operator_class_path": "airflow.providers.standard.operators.python.PythonOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_2", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_1.id.with.dots']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_2", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "sourceCode": { + "language": "python", + "sourceCode": "def do_nothing():\n pass\n", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/SourceCodeJobFacet.json\\#\\/\\$defs\\/SourceCodeJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "duration": "{{ result is number }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=task_2.*\") }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_3' in result and 'check_events_number' in result }}", + "executor_config": {}, + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[{'uri': 's3://bucket2/dir2/file2.txt', 'extra': None}]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "max_retry_delay": "42.0 seconds", + "multiple_outputs": false, + "operator_class": "PythonOperator", + "operator_class_path": "airflow.providers.standard.operators.python.PythonOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_2", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_1.id.with.dots']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_2", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=task_3.*\") }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_4' in result and 'section_1.task_5' in result }}", + "executor_config": {}, + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "multiple_outputs": false, + "operator_class": "EmptyOperator", + "operator_class_path": "airflow.providers.standard.operators.empty.EmptyOperator", + "outlets": "[{'uri': 's3://bucket/dir/file.txt', 'extra': None}]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_3", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_2']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_3", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "duration": "{{ result is number }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=task_3.*\") }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_4' in result and 'section_1.task_5' in result }}", + "executor_config": {}, + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "multiple_outputs": false, + "operator_class": "EmptyOperator", + "operator_class_path": "airflow.providers.standard.operators.empty.EmptyOperator", + "outlets": "[{'uri': 's3://bucket/dir/file.txt', 'extra': None}]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_3", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_2']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_3", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=task_4.*\") }}" + }, + "task": { + "deferrable": "{{ result == true }}", + "depends_on_past": false, + "downstream_task_ids": "['section_1.section_2.task_6']", + "executor_config": {}, + "external_dag_id": "external_dag_id", + "external_task_id": "external_task_id", + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "max_active_tis_per_dag": 7, + "max_active_tis_per_dagrun": 2, + "multiple_outputs": false, + "operator_class": "SomeCustomOperator", + "operator_class_path": "{{ result.endswith('openlineage_base_complex_dag.SomeCustomOperator') }}", + "outlets": "[]", + "owner": "owner3", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_4", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_3']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_4", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "duration": "{{ result is number }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=task_4.*\") }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "try_number": "{{ result is number }}" + }, + "task": { + "deferrable": "{{ result == true }}", + "depends_on_past": false, + "downstream_task_ids": "['section_1.section_2.task_6']", + "executor_config": {}, + "external_dag_id": "external_dag_id", + "external_task_id": "external_task_id", + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "max_active_tis_per_dag": 7, + "max_active_tis_per_dagrun": 2, + "multiple_outputs": false, + "operator_class": "SomeCustomOperator", + "operator_class_path": "{{ result.endswith('openlineage_base_complex_dag.SomeCustomOperator') }}", + "outlets": "[]", + "owner": "owner3", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_4", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_3']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_4", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=section_1.section_2.task_6.*\") }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "['check_events_number']", + "executor_config": {}, + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "multiple_outputs": false, + "operator_class": "EmptyOperator", + "operator_class_path": "airflow.providers.standard.operators.empty.EmptyOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "sla": "123.0 seconds", + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_group": { + "downstream_group_ids": "[]", + "downstream_task_ids": "[]", + "group_id": "section_2", + "prefix_group_id": true, + "tooltip": "group_tooltip", + "upstream_group_ids": "[]", + "upstream_task_ids": "[]" + }, + "task_id": "section_1.section_2.task_6", + "trigger_rule": "all_success", + "upstream_task_ids": "{{ 'task_4' in result and 'section_1.task_5' in result }}", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.section_1.section_2.task_6", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "duration": "{{ result is number }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_complex_dag\\/grid\\?dag_run_id=.*?\\&task_id=section_1.section_2.task_6.*\") }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "['check_events_number']", + "executor_config": {}, + "ignore_first_depends_on_past": "{{ result == true }}", + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "multiple_outputs": false, + "operator_class": "EmptyOperator", + "operator_class_path": "airflow.providers.standard.operators.empty.EmptyOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "sla": "123.0 seconds", + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_group": { + "downstream_group_ids": "[]", + "downstream_task_ids": "[]", + "group_id": "section_2", + "prefix_group_id": true, + "tooltip": "group_tooltip", + "upstream_group_ids": "[]", + "upstream_task_ids": "[]" + }, + "task_id": "section_1.section_2.task_6", + "trigger_rule": "all_success", + "upstream_task_ids": "{{ 'task_4' in result and 'section_1.task_5' in result }}", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.section_1.section_2.task_6", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_complex_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_complex_dag__af3.json new file mode 100644 index 0000000000000..cdd3fa38310e7 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_complex_dag__af3.json @@ -0,0 +1,1392 @@ +[ + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "airflowDagRun": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "DAG", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "airflow": { + "taskGroups": { + "section_1": { + "ui_color": "CornflowerBlue", + "ui_fgcolor": "#000", + "ui_label": "section_1" + }, + "section_1.section_2": { + "parent_group": "section_1", + "ui_color": "CornflowerBlue", + "ui_fgcolor": "#000", + "ui_label": "section_2" + }, + "section_1.section_2.section_3": { + "parent_group": "section_1.section_2", + "ui_color": "CornflowerBlue", + "ui_fgcolor": "#000", + "ui_label": "section_3" + } + }, + "taskTree": {}, + "tasks": { + "section_1.section_2.section_3.task_7": { + "downstream_task_ids": [ + "check_events_number" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.python.PythonOperator", + "task_group": "section_1.section_2.section_3", + "ui_color": "#ffefeb", + "ui_fgcolor": "#000", + "ui_label": "task_7" + }, + "section_1.section_2.task_6": { + "downstream_task_ids": [ + "check_events_number" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.empty.EmptyOperator", + "task_group": "section_1.section_2", + "ui_color": "#e8f7e4", + "ui_fgcolor": "#000", + "ui_label": "task_6" + }, + "section_1.task_5": { + "downstream_task_ids": [ + "section_1.section_2.task_6" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.python.PythonOperator", + "task_group": "section_1", + "ui_color": "#ffefeb", + "ui_fgcolor": "#000", + "ui_label": "task_5" + }, + "task_0": { + "downstream_task_ids": [], + "emits_ol_events": false, + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.empty.EmptyOperator", + "ui_color": "#e8f7e4", + "ui_fgcolor": "#000", + "ui_label": "task_0" + }, + "task_1.id.with.dots": { + "downstream_task_ids": "{{ result | sort == ['section_1.section_2.section_3.task_7', 'task_2'] }}", + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.bash.BashOperator", + "ui_color": "#f0ede4", + "ui_fgcolor": "#000", + "ui_label": "task_1.id.with.dots" + }, + "task_2": { + "downstream_task_ids": "{{ result | sort == ['check_events_number', 'task_3'] }}", + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.python.PythonOperator", + "ui_color": "#ffefeb", + "ui_fgcolor": "#000", + "ui_label": "task_2" + }, + "task_3": { + "downstream_task_ids": "{{ result | sort == ['section_1.task_5', 'task_4'] }}", + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.empty.EmptyOperator", + "ui_color": "#e8f7e4", + "ui_fgcolor": "#000", + "ui_label": "task_3" + }, + "task_4": { + "downstream_task_ids": [ + "section_1.section_2.task_6" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "{{ result.endswith('openlineage_base_complex_dag.SomeCustomOperator') }}", + "ui_color": "#f0ede4", + "ui_fgcolor": "#000", + "ui_label": "task_4" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/JobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_2' in result and 'section_1.section_2.section_3.task_7' in result }}", + "execution_timeout": "456.0 seconds", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "multiple_outputs": false, + "operator_class": "BashOperator", + "operator_class_path": "airflow.providers.standard.operators.bash.BashOperator", + "outlets": "[]", + "owner": "owner2", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_1.id.with.dots", + "trigger_rule": "all_success", + "upstream_task_ids": "[]", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_1.id.with.dots", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "sourceCode": { + "language": "bash", + "sourceCode": "exit 0;", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/SourceCodeJobFacet.json\\#\\/\\$defs\\/SourceCodeJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_2' in result and 'section_1.section_2.section_3.task_7' in result }}", + "execution_timeout": "456.0 seconds", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "multiple_outputs": false, + "operator_class": "BashOperator", + "operator_class_path": "airflow.providers.standard.operators.bash.BashOperator", + "outlets": "[]", + "owner": "owner2", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_1.id.with.dots", + "trigger_rule": "all_success", + "upstream_task_ids": "[]", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_1.id.with.dots", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_3' in result and 'check_events_number' in result }}", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[{'uri': 's3://bucket2/dir2/file2.txt', 'extra': {}}]", + "mapped": false, + "max_retry_delay": "42.0 seconds", + "multiple_outputs": false, + "operator_class": "PythonOperator", + "operator_class_path": "airflow.providers.standard.operators.python.PythonOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_2", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_1.id.with.dots']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_2", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "sourceCode": { + "language": "python", + "sourceCode": "def do_nothing():\n pass\n", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/SourceCodeJobFacet.json\\#\\/\\$defs\\/SourceCodeJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_3' in result and 'check_events_number' in result }}", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[{'uri': 's3://bucket2/dir2/file2.txt', 'extra': {}}]", + "mapped": false, + "max_retry_delay": "42.0 seconds", + "multiple_outputs": false, + "operator_class": "PythonOperator", + "operator_class_path": "airflow.providers.standard.operators.python.PythonOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_2", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_1.id.with.dots']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_2", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_4' in result and 'section_1.task_5' in result }}", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "multiple_outputs": false, + "operator_class": "EmptyOperator", + "operator_class_path": "airflow.providers.standard.operators.empty.EmptyOperator", + "outlets": "[{'uri': 's3://bucket/dir/file.txt', 'extra': {}}]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_3", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_2']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_3", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "{{ 'task_4' in result and 'section_1.task_5' in result }}", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "multiple_outputs": false, + "operator_class": "EmptyOperator", + "operator_class_path": "airflow.providers.standard.operators.empty.EmptyOperator", + "outlets": "[{'uri': 's3://bucket/dir/file.txt', 'extra': {}}]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_3", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_2']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_3", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "deferrable": "{{ result == true }}", + "depends_on_past": false, + "downstream_task_ids": "['section_1.section_2.task_6']", + "executor_config": {}, + "external_dag_id": "external_dag_id", + "external_task_id": "external_task_id", + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "max_active_tis_per_dag": 7, + "max_active_tis_per_dagrun": 2, + "multiple_outputs": false, + "operator_class": "SomeCustomOperator", + "operator_class_path": "{{ result.endswith('openlineage_base_complex_dag.SomeCustomOperator') }}", + "outlets": "[]", + "owner": "owner3", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_4", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_3']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_4", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "deferrable": "{{ result == true }}", + "depends_on_past": false, + "downstream_task_ids": "['section_1.section_2.task_6']", + "executor_config": {}, + "external_dag_id": "external_dag_id", + "external_task_id": "external_task_id", + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "max_active_tis_per_dag": 7, + "max_active_tis_per_dagrun": 2, + "multiple_outputs": false, + "operator_class": "SomeCustomOperator", + "operator_class_path": "{{ result.endswith('openlineage_base_complex_dag.SomeCustomOperator') }}", + "outlets": "[]", + "owner": "owner3", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "task_4", + "trigger_rule": "all_success", + "upstream_task_ids": "['task_3']", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.task_4", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "['check_events_number']", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "multiple_outputs": false, + "operator_class": "EmptyOperator", + "operator_class_path": "airflow.providers.standard.operators.empty.EmptyOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "run_as_user": "some_user", + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_group": { + "downstream_group_ids": "[]", + "downstream_task_ids": "[]", + "group_id": "section_2", + "prefix_group_id": "{{ result == true }}", + "tooltip": "group_tooltip", + "upstream_group_ids": "[]", + "upstream_task_ids": "[]" + }, + "task_id": "section_1.section_2.task_6", + "trigger_rule": "all_success", + "upstream_task_ids": "{{ 'task_4' in result and 'section_1.task_5' in result }}", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.section_1.section_2.task_6", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "documentation": { + "description": "OpenLineage complex DAG description", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/DocumentationJobFacet.json\\#\\/\\$defs\\/DocumentationJobFacet\") }}" + }, + "ownership": { + "owners": "{{ result | map(attribute='name') | list | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_complex_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_complex_dag", + "description": "OpenLineage complex DAG description", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_complex_dag.py') }}", + "owner": "{{ result.split(', ') | sort == ['airflow', 'owner1', 'owner2', 'owner3'] }}", + "owner_links": { + "airflow": "https://airflow.apache.org/" + }, + "start_date": "{{ is_datetime(result) }}", + "tags": "{{ result[1:-1].split(', ') | sort == ['\"with\\'quote\"', \"'first'\", \"'second@'\"] }}", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_complex_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "['check_events_number']", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "multiple_outputs": false, + "operator_class": "EmptyOperator", + "operator_class_path": "airflow.providers.standard.operators.empty.EmptyOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "run_as_user": "some_user", + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_group": { + "downstream_group_ids": "[]", + "downstream_task_ids": "[]", + "group_id": "section_2", + "prefix_group_id": "{{ result == true }}", + "tooltip": "group_tooltip", + "upstream_group_ids": "[]", + "upstream_task_ids": "[]" + }, + "task_id": "section_1.section_2.task_6", + "trigger_rule": "all_success", + "upstream_task_ids": "{{ 'task_4' in result and 'section_1.task_5' in result }}", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_complex_dag.section_1.section_2.task_6", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_simple_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_simple_dag__af2.json new file mode 100644 index 0000000000000..972048659eca9 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_simple_dag__af2.json @@ -0,0 +1,338 @@ +[ + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "airflowDagRun": { + "dag": { + "dag_id": "openlineage_base_simple_dag", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_simple_dag.py') }}", + "owner": "airflow", + "owner_links": {}, + "start_date": "{{ is_datetime(result) }}", + "tags": "[]", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_simple_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true or result == false }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^2\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "DAG", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "ownership": { + "owners": [ + { + "name": "airflow" + } + ], + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "airflow": { + "taskGroups": {}, + "taskTree": {}, + "tasks": { + "do_nothing_task": { + "downstream_task_ids": [ + "check_events" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.python.PythonOperator", + "ui_color": "#ffefeb", + "ui_fgcolor": "#000", + "ui_label": "do_nothing_task" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/JobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_simple_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_simple_dag", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_simple_dag.py') }}", + "owner": "airflow", + "owner_links": {}, + "start_date": "{{ is_datetime(result) }}", + "tags": "[]", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_simple_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true or result == false }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_simple_dag\\/grid\\?dag_run_id=.*?\\&task_id=do_nothing_task.*\") }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "['check_events']", + "executor_config": {}, + "ignore_first_depends_on_past": true, + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "multiple_outputs": false, + "operator_class": "PythonOperator", + "operator_class_path": "airflow.providers.standard.operators.python.PythonOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "do_nothing_task", + "trigger_rule": "all_success", + "upstream_task_ids": "[]", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^2\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag.do_nothing_task", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "ownership": { + "owners": [ + { + "name": "airflow" + } + ], + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "sourceCode": { + "language": "python", + "sourceCode": "def do_nothing():\n pass\n", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/SourceCodeJobFacet.json\\#\\/\\$defs\\/SourceCodeJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_simple_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_simple_dag", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_simple_dag.py') }}", + "owner": "airflow", + "owner_links": {}, + "start_date": "{{ is_datetime(result) }}", + "tags": "[]", + "timetable": {} + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_simple_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "external_trigger": "{{ result == true or result == false }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "duration": "{{ result is number }}", + "log_url": "{{ regex_match(result, \"^http[s]?:\\/\\/.*\\/dags\\/openlineage_base_simple_dag\\/grid\\?dag_run_id=.*?\\&task_id=do_nothing_task.*\") }}", + "pool": "{{ result is string }}", + "queued_dttm": "{{ is_datetime(result) }}", + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "['check_events']", + "executor_config": {}, + "ignore_first_depends_on_past": true, + "inlets": "[]", + "is_setup": false, + "is_teardown": false, + "mapped": false, + "multiple_outputs": false, + "operator_class": "PythonOperator", + "operator_class_path": "airflow.providers.standard.operators.python.PythonOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "do_nothing_task", + "trigger_rule": "all_success", + "upstream_task_ids": "[]", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^2\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag.do_nothing_task", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "sourceCode": { + "language": "python", + "sourceCode": "def do_nothing():\n pass\n", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/SourceCodeJobFacet.json\\#\\/\\$defs\\/SourceCodeJobFacet\") }}" + } + } + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_simple_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_simple_dag__af3.json new file mode 100644 index 0000000000000..4dcfd63b92ceb --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_base_simple_dag__af3.json @@ -0,0 +1,330 @@ +[ + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "airflowDagRun": { + "dag": { + "dag_id": "openlineage_base_simple_dag", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_simple_dag.py') }}", + "owner": "airflow", + "owner_links": {}, + "start_date": "{{ is_datetime(result) }}", + "tags": "[]", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_simple_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^3\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "DAG", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "ownership": { + "owners": [ + { + "name": "airflow" + } + ], + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "airflow": { + "taskGroups": {}, + "taskTree": {}, + "tasks": { + "do_nothing_task": { + "downstream_task_ids": [ + "check_events" + ], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.python.PythonOperator", + "ui_color": "#ffefeb", + "ui_fgcolor": "#000", + "ui_label": "do_nothing_task" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/JobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_simple_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_simple_dag", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_simple_dag.py') }}", + "owner": "airflow", + "owner_links": {}, + "start_date": "{{ is_datetime(result) }}", + "tags": "[]", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_simple_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "['check_events']", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "multiple_outputs": false, + "operator_class": "PythonOperator", + "operator_class_path": "airflow.providers.standard.operators.python.PythonOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "do_nothing_task", + "trigger_rule": "all_success", + "upstream_task_ids": "[]", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^3\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag.do_nothing_task", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "ownership": { + "owners": [ + { + "name": "airflow" + } + ], + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "sourceCode": { + "language": "python", + "sourceCode": "def do_nothing():\n pass\n", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/SourceCodeJobFacet.json\\#\\/\\$defs\\/SourceCodeJobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "parent": { + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + }, + "root": { + "job": { + "name": "openlineage_base_simple_dag", + "namespace": "{{ result is string }}" + }, + "run": { + "runId": "{{ is_uuid(result) }}" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ParentRunFacet.json\\#\\/\\$defs\\/ParentRunFacet$\") }}" + }, + "airflow": { + "dag": { + "dag_id": "openlineage_base_simple_dag", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_base_simple_dag.py') }}", + "owner": "airflow", + "owner_links": {}, + "start_date": "{{ is_datetime(result) }}", + "tags": "[]", + "timetable": {}, + "timetable_summary": "None" + }, + "dagRun": { + "conf": {}, + "dag_id": "openlineage_base_simple_dag", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_after": "{{ is_datetime(result) }}", + "run_id": "{{ result is string }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "taskInstance": { + "try_number": "{{ result is number }}" + }, + "task": { + "depends_on_past": false, + "downstream_task_ids": "['check_events']", + "executor_config": {}, + "ignore_first_depends_on_past": false, + "inlets": "[]", + "mapped": false, + "multiple_outputs": false, + "operator_class": "PythonOperator", + "operator_class_path": "airflow.providers.standard.operators.python.PythonOperator", + "outlets": "[]", + "owner": "airflow", + "priority_weight": 1, + "queue": "{{ result is string }}", + "retries": "{{ result is number }}", + "retry_exponential_backoff": false, + "task_id": "do_nothing_task", + "trigger_rule": "all_success", + "upstream_task_ids": "[]", + "wait_for_downstream": false, + "wait_for_past_depends_before_skipping": false + }, + "taskUuid": "{{ is_uuid(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^3\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_base_simple_dag.do_nothing_task", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "TASK", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "sourceCode": { + "language": "python", + "sourceCode": "def do_nothing():\n pass\n", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/SourceCodeJobFacet.json\\#\\/\\$defs\\/SourceCodeJobFacet\") }}" + } + } + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_defer_simple_dag.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_defer_simple_dag.json new file mode 100644 index 0000000000000..d3604f916d0c6 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_defer_simple_dag.json @@ -0,0 +1,32 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "taskInstance": { + "try_number": 1 + } + } + } + }, + "job": { + "name": "openlineage_defer_simple_dag.wait" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "taskInstance": { + "try_number": 1 + } + } + } + }, + "job": { + "name": "openlineage_defer_simple_dag.wait" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_mapped_simple_dag.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_mapped_simple_dag.json new file mode 100644 index 0000000000000..be04d491a7e54 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_mapped_simple_dag.json @@ -0,0 +1,38 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "task": { + "mapped": "{{ result == true }}" + }, + "taskInstance": { + "map_index": 1 + } + } + } + }, + "job": { + "name": "openlineage_mapped_simple_dag.add_one" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "task": { + "mapped": "{{ result == true }}" + }, + "taskInstance": { + "map_index": 1 + } + } + } + }, + "job": { + "name": "openlineage_mapped_simple_dag.add_one" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_asset_or_time_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_asset_or_time_dag__af2.json new file mode 100644 index 0000000000000..ffcd43d754686 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_asset_or_time_dag__af2.json @@ -0,0 +1,188 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "timetable": { + "__type": "airflow.timetables.trigger.CronTriggerTimetable", + "__var": { + "expression": "21 13 29 2 4", + "timezone": "UTC", + "interval": 0.0 + } + }, + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "a": 1 + } + }, + { + "__type": "dataset", + "uri": "s3://bucket2/file.txt" + } + ] + }, + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket3/file.txt" + }, + { + "__type": "dataset", + "uri": "s3://bucket4/file.txt", + "extra": { + "b": 2 + } + } + ] + } + ] + } + }, + "schedule_interval": "Dataset or 21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_asset_or_time_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "timetable": { + "__type": "airflow.timetables.trigger.CronTriggerTimetable", + "__var": { + "expression": "21 13 29 2 4", + "timezone": "UTC", + "interval": 0.0 + } + }, + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "a": 1 + } + }, + { + "__type": "dataset", + "uri": "s3://bucket2/file.txt" + } + ] + }, + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket3/file.txt" + }, + { + "__type": "dataset", + "uri": "s3://bucket4/file.txt", + "extra": { + "b": 2 + } + } + ] + } + ] + } + }, + "schedule_interval": "Dataset or 21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_asset_or_time_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "timetable": { + "__type": "airflow.timetables.trigger.CronTriggerTimetable", + "__var": { + "expression": "21 13 29 2 4", + "timezone": "UTC", + "interval": 0.0 + } + }, + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "a": 1 + } + }, + { + "__type": "dataset", + "uri": "s3://bucket2/file.txt" + } + ] + }, + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket3/file.txt" + }, + { + "__type": "dataset", + "uri": "s3://bucket4/file.txt", + "extra": { + "b": 2 + } + } + ] + } + ] + } + }, + "schedule_interval": "Dataset or 21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_asset_or_time_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_asset_or_time_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_asset_or_time_dag__af3.json new file mode 100644 index 0000000000000..509cc731e52d1 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_asset_or_time_dag__af3.json @@ -0,0 +1,215 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "timetable": { + "__type": "airflow.timetables.trigger.CronTriggerTimetable", + "__var": { + "expression": "21 13 29 2 4", + "timezone": "UTC", + "interval": 0.0, + "run_immediately": false + } + }, + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + }, + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket3/file.txt", + "name": "s3://bucket3/file.txt", + "group": "asset", + "extra": {} + }, + { + "__type": "asset", + "uri": "s3://bucket4/file.txt", + "name": "s3://bucket4/file.txt", + "group": "asset", + "extra": {"b": 2} + } + ] + } + ] + } + }, + "timetable_summary": "Asset or 21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_asset_or_time_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "timetable": { + "__type": "airflow.timetables.trigger.CronTriggerTimetable", + "__var": { + "expression": "21 13 29 2 4", + "timezone": "UTC", + "interval": 0.0, + "run_immediately": false + } + }, + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + }, + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket3/file.txt", + "name": "s3://bucket3/file.txt", + "group": "asset", + "extra": {} + }, + { + "__type": "asset", + "uri": "s3://bucket4/file.txt", + "name": "s3://bucket4/file.txt", + "group": "asset", + "extra": {"b": 2} + } + ] + } + ] + } + }, + "timetable_summary": "Asset or 21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_asset_or_time_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "timetable": { + "__type": "airflow.timetables.trigger.CronTriggerTimetable", + "__var": { + "expression": "21 13 29 2 4", + "timezone": "UTC", + "interval": 0.0, + "run_immediately": false + } + }, + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + }, + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket3/file.txt", + "name": "s3://bucket3/file.txt", + "group": "asset", + "extra": {} + }, + { + "__type": "asset", + "uri": "s3://bucket4/file.txt", + "name": "s3://bucket4/file.txt", + "group": "asset", + "extra": {"b": 2} + } + ] + } + ] + } + }, + "timetable_summary": "Asset or 21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_asset_or_time_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_cron_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_cron_dag__af2.json new file mode 100644 index 0000000000000..1a476c8b72026 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_cron_dag__af2.json @@ -0,0 +1,59 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "expression": "21 13 29 2 4", + "timezone": "UTC" + }, + "schedule_interval": "21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_cron_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "expression": "21 13 29 2 4", + "timezone": "UTC" + }, + "schedule_interval": "21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_cron_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "expression": "21 13 29 2 4", + "timezone": "UTC" + }, + "schedule_interval": "21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_cron_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_cron_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_cron_dag__af3.json new file mode 100644 index 0000000000000..cc24cdc65c1ce --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_cron_dag__af3.json @@ -0,0 +1,59 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "expression": "21 13 29 2 4", + "timezone": "UTC" + }, + "timetable_summary": "21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_cron_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "expression": "21 13 29 2 4", + "timezone": "UTC" + }, + "timetable_summary": "21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_cron_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "expression": "21 13 29 2 4", + "timezone": "UTC" + }, + "timetable_summary": "21 13 29 2 4" + } + } + } + }, + "job": { + "name": "openlineage_schedule_cron_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_complex_assets_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_complex_assets_dag__af2.json new file mode 100644 index 0000000000000..fe51488c7066f --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_complex_assets_dag__af2.json @@ -0,0 +1 @@ +[] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_complex_assets_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_complex_assets_dag__af3.json new file mode 100644 index 0000000000000..003e149339dec --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_complex_assets_dag__af3.json @@ -0,0 +1,206 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + }, + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket3/file.txt", + "name": "s3://bucket3/file.txt", + "group": "asset", + "extra": {} + }, + { + "__type": "asset", + "uri": "s3://bucket4/file.txt", + "name": "s3://bucket4/file.txt", + "group": "asset", + "extra": {"b": 2} + } + ] + } + ] + } + ] + + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_complex_assets_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + }, + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket3/file.txt", + "name": "s3://bucket3/file.txt", + "group": "asset", + "extra": {} + }, + { + "__type": "asset", + "uri": "s3://bucket4/file.txt", + "name": "s3://bucket4/file.txt", + "group": "asset", + "extra": {"b": 2} + } + ] + } + ] + } + ] + + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_complex_assets_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + }, + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket3/file.txt", + "name": "s3://bucket3/file.txt", + "group": "asset", + "extra": {} + }, + { + "__type": "asset", + "uri": "s3://bucket4/file.txt", + "name": "s3://bucket4/file.txt", + "group": "asset", + "extra": {"b": 2} + } + ] + } + ] + } + ] + + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_complex_assets_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_multiple_assets_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_multiple_assets_dag__af2.json new file mode 100644 index 0000000000000..ca47fca149fee --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_multiple_assets_dag__af2.json @@ -0,0 +1,101 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "a": 1 + } + }, + { + "__type": "dataset", + "uri": "s3://bucket2/file.txt" + } + ] + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_multiple_assets_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "a": 1 + } + }, + { + "__type": "dataset", + "uri": "s3://bucket2/file.txt" + } + ] + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_multiple_assets_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "a": 1 + } + }, + { + "__type": "dataset", + "uri": "s3://bucket2/file.txt" + } + ] + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_multiple_assets_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_multiple_assets_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_multiple_assets_dag__af3.json new file mode 100644 index 0000000000000..214e4b5059701 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_multiple_assets_dag__af3.json @@ -0,0 +1,116 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_multiple_assets_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_multiple_assets_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_multiple_assets_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_single_asset_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_single_asset_dag__af2.json new file mode 100644 index 0000000000000..98c2c25b25404 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_single_asset_dag__af2.json @@ -0,0 +1,89 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "some_extra": 123 + } + } + ] + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_single_asset_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "some_extra": 123 + } + } + ] + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_single_asset_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "some_extra": 123 + } + } + ] + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_single_asset_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_single_asset_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_single_asset_dag__af3.json new file mode 100644 index 0000000000000..ef95e1a03394d --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_list_single_asset_dag__af3.json @@ -0,0 +1,95 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "some_extra": 123 + } + } + ] + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_single_asset_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "some_extra": 123 + } + } + ] + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_single_asset_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "some_extra": 123 + } + } + ] + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_list_single_asset_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_multiple_assets_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_multiple_assets_dag__af2.json new file mode 100644 index 0000000000000..783091cfc0d12 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_multiple_assets_dag__af2.json @@ -0,0 +1,164 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "a": 1 + } + }, + { + "__type": "dataset", + "uri": "s3://bucket2/file.txt" + } + ] + }, + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket3/file.txt" + }, + { + "__type": "dataset", + "uri": "s3://bucket4/file.txt", + "extra": { + "b": 2 + } + } + ] + } + ] + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_multiple_assets_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "a": 1 + } + }, + { + "__type": "dataset", + "uri": "s3://bucket2/file.txt" + } + ] + }, + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket3/file.txt" + }, + { + "__type": "dataset", + "uri": "s3://bucket4/file.txt", + "extra": { + "b": 2 + } + } + ] + } + ] + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_multiple_assets_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset_all", + "objects": [ + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "a": 1 + } + }, + { + "__type": "dataset", + "uri": "s3://bucket2/file.txt" + } + ] + }, + { + "__type": "dataset_any", + "objects": [ + { + "__type": "dataset", + "uri": "s3://bucket3/file.txt" + }, + { + "__type": "dataset", + "uri": "s3://bucket4/file.txt", + "extra": { + "b": 2 + } + } + ] + } + ] + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_multiple_assets_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_multiple_assets_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_multiple_assets_dag__af3.json new file mode 100644 index 0000000000000..40ae29b228e1b --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_multiple_assets_dag__af3.json @@ -0,0 +1,188 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + }, + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket3/file.txt", + "name": "s3://bucket3/file.txt", + "group": "asset", + "extra": {} + }, + { + "__type": "asset", + "uri": "s3://bucket4/file.txt", + "name": "s3://bucket4/file.txt", + "group": "asset", + "extra": {"b": 2} + } + ] + } + ] + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_multiple_assets_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + }, + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket3/file.txt", + "name": "s3://bucket3/file.txt", + "group": "asset", + "extra": {} + }, + { + "__type": "asset", + "uri": "s3://bucket4/file.txt", + "name": "s3://bucket4/file.txt", + "group": "asset", + "extra": {"b": 2} + } + ] + } + ] + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_multiple_assets_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset_all", + "objects": [ + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "a": 1 + } + }, + { + "__type": "asset", + "uri": "s3://bucket2/file.txt", + "name": "s3://bucket2/file.txt", + "group": "asset", + "extra": {} + } + ] + }, + { + "__type": "asset_any", + "objects": [ + { + "__type": "asset", + "uri": "s3://bucket3/file.txt", + "name": "s3://bucket3/file.txt", + "group": "asset", + "extra": {} + }, + { + "__type": "asset", + "uri": "s3://bucket4/file.txt", + "name": "s3://bucket4/file.txt", + "group": "asset", + "extra": {"b": 2} + } + ] + } + ] + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_multiple_assets_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_single_asset_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_single_asset_dag__af2.json new file mode 100644 index 0000000000000..f9b3e2411400c --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_single_asset_dag__af2.json @@ -0,0 +1,74 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "some_extra": 123 + } + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_single_asset_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "some_extra": 123 + } + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_single_asset_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "dataset_condition": { + "__type": "dataset", + "uri": "s3://bucket/file.txt", + "extra": { + "some_extra": 123 + } + } + }, + "schedule_interval": "Dataset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_single_asset_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_single_asset_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_single_asset_dag__af3.json new file mode 100644 index 0000000000000..98b19a7bc525c --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_single_asset_dag__af3.json @@ -0,0 +1,80 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "some_extra": 123 + } + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_single_asset_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "some_extra": 123 + } + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_single_asset_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "asset_condition": { + "__type": "asset", + "uri": "s3://bucket/file.txt", + "name": "s3://bucket/file.txt", + "group": "asset", + "extra": { + "some_extra": 123 + } + } + }, + "timetable_summary": "Asset" + } + } + } + }, + "job": { + "name": "openlineage_schedule_single_asset_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_timetable_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_timetable_dag__af2.json new file mode 100644 index 0000000000000..2269e8c6117b3 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_timetable_dag__af2.json @@ -0,0 +1,71 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "event_dates": [ + "2095-03-03 08:27:00-06:00", + "2095-03-17 08:27:00-05:00", + "2095-03-22 20:50:00-05:00" + ], + "restrict_to_events": false + }, + "schedule_interval": "3 events" + } + } + } + }, + "job": { + "name": "openlineage_schedule_timetable_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "event_dates": [ + "2095-03-03 08:27:00-06:00", + "2095-03-17 08:27:00-05:00", + "2095-03-22 20:50:00-05:00" + ], + "restrict_to_events": false + }, + "schedule_interval": "My Team's Baseball Games" + } + } + } + }, + "job": { + "name": "openlineage_schedule_timetable_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "event_dates": [ + "2095-03-03 08:27:00-06:00", + "2095-03-17 08:27:00-05:00", + "2095-03-22 20:50:00-05:00" + ], + "restrict_to_events": false + }, + "schedule_interval": "My Team's Baseball Games" + } + } + } + }, + "job": { + "name": "openlineage_schedule_timetable_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_timetable_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_timetable_dag__af3.json new file mode 100644 index 0000000000000..537e6a22d9e9a --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_schedule_timetable_dag__af3.json @@ -0,0 +1,71 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dag": { + "timetable": { + "event_dates": [ + "2095-03-03T08:27:00-06:00", + "2095-03-17T08:27:00-05:00", + "2095-03-22T20:50:00-05:00" + ], + "restrict_to_events": false + }, + "timetable_summary": "3 events" + } + } + } + }, + "job": { + "name": "openlineage_schedule_timetable_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "event_dates": [ + "2095-03-03T08:27:00-06:00", + "2095-03-17T08:27:00-05:00", + "2095-03-22T20:50:00-05:00" + ], + "restrict_to_events": false + }, + "timetable_summary": "My Team's Baseball Games" + } + } + } + }, + "job": { + "name": "openlineage_schedule_timetable_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "dag": { + "timetable": { + "event_dates": [ + "2095-03-03T08:27:00-06:00", + "2095-03-17T08:27:00-05:00", + "2095-03-22T20:50:00-05:00" + ], + "restrict_to_events": false + }, + "timetable_summary": "My Team's Baseball Games" + } + } + } + }, + "job": { + "name": "openlineage_schedule_timetable_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_task_groups_dag.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_task_groups_dag.json new file mode 100644 index 0000000000000..c1b2124c6b715 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_task_groups_dag.json @@ -0,0 +1,94 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "task": { + "task_group": { + "downstream_group_ids": "['tg2']", + "downstream_task_ids": "[]", + "group_id": "tg1", + "prefix_group_id": true, + "tooltip": "", + "upstream_group_ids": "[]", + "upstream_task_ids": "['task_0']" + } + } + } + } + }, + "job": { + "name": "openlineage_task_groups_dag.tg1.task_1" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "task": { + "task_group": { + "downstream_group_ids": "['tg2']", + "downstream_task_ids": "[]", + "group_id": "tg1", + "prefix_group_id": true, + "tooltip": "", + "upstream_group_ids": "[]", + "upstream_task_ids": "['task_0']" + } + } + } + } + }, + "job": { + "name": "openlineage_task_groups_dag.tg1.task_1" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "task": { + "task_group": { + "downstream_group_ids": "[]", + "downstream_task_ids": "['check_events']", + "group_id": "tg2", + "prefix_group_id": true, + "tooltip": "", + "upstream_group_ids": "['tg1']", + "upstream_task_ids": "['tg1.task_1']" + } + } + } + } + }, + "job": { + "name": "openlineage_task_groups_dag.tg2.task_2" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "task": { + "task_group": { + "downstream_group_ids": "[]", + "downstream_task_ids": "['check_events']", + "group_id": "tg2", + "prefix_group_id": true, + "tooltip": "", + "upstream_group_ids": "['tg1']", + "upstream_task_ids": "['tg1.task_1']" + } + } + } + } + }, + "job": { + "name": "openlineage_task_groups_dag.tg2.task_2" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_trigger_dag.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_trigger_dag.json new file mode 100644 index 0000000000000..d39207d006fd5 --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_trigger_dag.json @@ -0,0 +1,188 @@ +[ + { + "eventType": "START", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "airflowDagRun": { + "dag": { + "dag_id": "openlineage_trigger_dag_child__notrigger", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_trigger_dag.py') }}", + "owner": "airflow", + "owner_links": {}, + "start_date": "{{ is_datetime(result) }}", + "tags": "[]", + "timetable": {} + }, + "dagRun": { + "conf": { + "some_config": "value1" + }, + "dag_id": "openlineage_trigger_dag_child__notrigger", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result.startswith('openlineage_trigger_dag_triggering_child') }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}" + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "nominalTime": { + "nominalEndTime": "{{ is_datetime(result) }}", + "nominalStartTime": "{{ is_datetime(result) }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/NominalTimeRunFacet.json\\#\\/\\$defs\\/NominalTimeRunFacet$\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_trigger_dag_child__notrigger", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "DAG", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + }, + "ownership": { + "owners": [ + { + "name": "airflow" + } + ], + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/OwnershipJobFacet.json\\#\\/\\$defs\\/OwnershipJobFacet\") }}" + }, + "airflow": { + "taskGroups": {}, + "taskTree": {}, + "tasks": { + "do_nothing_task": { + "downstream_task_ids": [], + "emits_ol_events": "{{ result == true }}", + "is_setup": false, + "is_teardown": false, + "operator": "airflow.providers.standard.operators.bash.BashOperator", + "ui_label": "do_nothing_task" + } + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/JobFacet\") }}" + } + } + } + }, + { + "eventType": "COMPLETE", + "eventTime": "{{ is_datetime(result) }}", + "producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunEvent$\") }}", + "inputs": [], + "outputs": [], + "run": { + "runId": "{{ is_uuid(result) }}", + "facets": { + "airflowState": { + "dagRunState": "success", + "tasksState": { + "do_nothing_task": "success" + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "airflowDagRun": { + "dag": { + "dag_id": "openlineage_trigger_dag_child__notrigger", + "fileloc": "{{ result.endswith('openlineage/example_openlineage_trigger_dag.py') }}", + "owner": "airflow", + "owner_links": {}, + "start_date": "{{ is_datetime(result) }}", + "tags": "[]", + "timetable": {} + }, + "dagRun": { + "conf": { + "some_config": "value1" + }, + "dag_id": "openlineage_trigger_dag_child__notrigger", + "data_interval_end": "{{ is_datetime(result) }}", + "data_interval_start": "{{ is_datetime(result) }}", + "logical_date": "{{ is_datetime(result) }}", + "run_id": "{{ result.startswith('openlineage_trigger_dag_triggering_child') }}", + "run_type": "manual", + "start_date": "{{ is_datetime(result) }}", + "duration": "{{ result is number }}" + }, + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/[\\d-]+\\/OpenLineage.json\\#\\/\\$defs\\/RunFacet\") }}" + }, + "processing_engine": { + "name": "Airflow", + "openlineageAdapterVersion": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "version": "{{ regex_match(result, \"^[\\d]+\\.[\\d]+\\.[\\d]+.*\") }}", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/ProcessingEngineRunFacet.json\\#\\/\\$defs\\/ProcessingEngineRunFacet$\") }}" + } + } + }, + "job": { + "namespace": "{{ result is string }}", + "name": "openlineage_trigger_dag_child__notrigger", + "facets": { + "jobType": { + "integration": "AIRFLOW", + "jobType": "DAG", + "processingType": "BATCH", + "_producer": "{{ regex_match(result, \"^https:\\/\\/github.com/apache/airflow/tree/providers-openlineage\\/[\\d]+\\.[\\d]+\\.[\\d]+.*$\") }}", + "_schemaURL": "{{ regex_match(result, \"^https:\\/\\/openlineage.io\\/spec\\/facets\\/[\\d-]+\\/JobTypeJobFacet.json\\#\\/\\$defs\\/JobTypeJobFacet\") }}" + } + } + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "task": { + "trigger_dag_id": "openlineage_trigger_dag_child__notrigger" + } + } + } + }, + "job": { + "name": "openlineage_trigger_dag.trigger_dagrun" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "task": { + "trigger_dag_id": "openlineage_trigger_dag_child__notrigger" + } + } + } + }, + "job": { + "name": "openlineage_trigger_dag.trigger_dagrun" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_versioned_dag__af2.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_versioned_dag__af2.json new file mode 100644 index 0000000000000..fe51488c7066f --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_versioned_dag__af2.json @@ -0,0 +1 @@ +[] diff --git a/providers/openlineage/tests/system/openlineage/expected_events/openlineage_versioned_dag__af3.json b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_versioned_dag__af3.json new file mode 100644 index 0000000000000..09ac861c065be --- /dev/null +++ b/providers/openlineage/tests/system/openlineage/expected_events/openlineage_versioned_dag__af3.json @@ -0,0 +1,53 @@ +[ + { + "eventType": "START", + "run": { + "facets": { + "airflowDagRun": { + "dagRun": { + "dag_id": "openlineage_versioned_dag", + "dag_bundle_name": "{{ result is string }}", + "dag_bundle_version": "{{ result is string }}", + "dag_version_id": "{{ is_uuid(result) }}", + "dag_version_number": "{{ result is number }}" + } + } + } + }, + "job": { + "name": "openlineage_versioned_dag" + } + }, + { + "eventType": "START", + "run": { + "facets": { + "airflow": { + "taskInstance": { + "dag_bundle_name": "{{ result is string }}", + "dag_bundle_version": "{{ result is string }}" + } + } + } + }, + "job": { + "name": "openlineage_versioned_dag.do_nothing_task" + } + }, + { + "eventType": "COMPLETE", + "run": { + "facets": { + "airflow": { + "taskInstance": { + "dag_bundle_name": "{{ result is string }}", + "dag_bundle_version": "{{ result is string }}" + } + } + } + }, + "job": { + "name": "openlineage_versioned_dag.do_nothing_task" + } + } +] diff --git a/providers/openlineage/tests/system/openlineage/operator.py b/providers/openlineage/tests/system/openlineage/operator.py index d742037ea2d24..353959c32307b 100644 --- a/providers/openlineage/tests/system/openlineage/operator.py +++ b/providers/openlineage/tests/system/openlineage/operator.py @@ -222,9 +222,11 @@ def __init__( def execute(self, context: Context) -> None: if self.file_path is not None: self.event_templates = {} + self.log.info("Reading OpenLineage event templates from file `%s`", self.file_path) with open(self.file_path) as f: # type: ignore[arg-type] events = json.load(f) for event in events: + # Just a single event per job and event type is loaded as this is the most common scenario key = event["job"]["name"] + ".event." + event["eventType"].lower() self.event_templates[key] = event try: @@ -241,7 +243,8 @@ def execute(self, context: Context) -> None: raise ValueError(f"No event for key {key}") if len(actual_events) != 1 and not self.multiple_events: raise ValueError(f"Expected one event for key {key}, got {len(actual_events)}") - if not match(template, json.loads(actual_events[0]), self.env): + # Last event is checked against the template, this will allow to f.e. check change in try_num + if not match(template, json.loads(actual_events[-1]), self.env): raise ValueError("Event received does not match one specified in test") finally: if self.delete: